diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index f216830327..8b80dc0a9f 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -42,13 +42,11 @@ jobs: toolchain: stable override: true - - uses: Swatinem/rust-cache@v2 - - name: build and lint with clippy run: cargo clippy --features azure,datafusion,s3,gcs,glue --tests - name: Spot-check build for native-tls features - run: cargo clippy --no-default-features --features azure,datafusion,s3-native-tls,gcs,glue-native-tls --tests + run: cargo clippy --no-default-features --features azure,datafusion,s3-native-tls,gcs,glue --tests - name: Check docs run: cargo doc --features azure,datafusion,s3,gcs,glue @@ -82,8 +80,6 @@ jobs: toolchain: "stable" override: true - - uses: Swatinem/rust-cache@v2 - - name: Run tests run: cargo test --verbose --features datafusion,azure @@ -118,22 +114,6 @@ jobs: toolchain: stable override: true - # - uses: actions/setup-java@v3 - # with: - # distribution: "zulu" - # java-version: "17" - - # - uses: beyondstorage/setup-hdfs@master - # with: - # hdfs-version: "3.3.2" - - # - name: Set Hadoop env - # run: | - # echo "CLASSPATH=$CLASSPATH:`hadoop classpath --glob`" >> $GITHUB_ENV - # echo "LD_LIBRARY_PATH=$JAVA_HOME/lib/server" >> $GITHUB_ENV - - - uses: Swatinem/rust-cache@v2 - - name: Start emulated services run: docker-compose up -d @@ -144,25 +124,3 @@ jobs: - name: Run tests with native-tls run: | cargo test --no-default-features --features integration_test,s3-native-tls,datafusion - - parquet2_test: - runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=line-tables-only" - CARGO_INCREMENTAL: 0 - - steps: - - uses: actions/checkout@v3 - - - name: Install minimal stable with clippy and rustfmt - uses: actions-rs/toolchain@v1 - with: - profile: default - toolchain: stable - override: true - - - uses: Swatinem/rust-cache@v2 - - - name: Run tests - working-directory: crates/deltalake-core - run: cargo test --no-default-features --features=parquet2 diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index e96be656f6..079cd66fcc 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -1,4 +1,4 @@ -name: Build documentation +name: Build (and maybe release) the documentation on: pull_request: diff --git a/.github/workflows/docs_release.yml b/.github/workflows/docs_release.yml new file mode 100644 index 0000000000..c1d23f5fc5 --- /dev/null +++ b/.github/workflows/docs_release.yml @@ -0,0 +1,26 @@ +name: Release documentation + +on: + pull_request: + types: + - closed + branches: [main] + paths: + - docs/** + - mkdocs.yml + +jobs: + release-docs: + if: github.event.pull_request.merged == true + permissions: + contents: write + runs-on: ubuntu-latest + steps: + - name: Trigger the docs release event + uses: peter-evans/repository-dispatch@v2 + with: + event-type: release-docs + client-payload: > + { + "tag": "${{ github.ref_name }}" + } diff --git a/.github/workflows/python_release.yml b/.github/workflows/python_release.yml index 0214717cc8..48611bacb4 100644 --- a/.github/workflows/python_release.yml +++ b/.github/workflows/python_release.yml @@ -1,4 +1,4 @@ -name: Release to PyPI +name: Release to PyPI and documentation on: push: @@ -103,6 +103,8 @@ jobs: release-pypi-mac, release-pypi-windows, ] + permissions: + contents: write runs-on: ubuntu-latest steps: - name: Trigger the docs release event diff --git a/.gitignore b/.gitignore index ca0576b47c..84fc17c5f2 100644 --- a/.gitignore +++ b/.gitignore @@ -11,6 +11,7 @@ tlaplus/*.toolbox/*/[0-9]*-[0-9]*-[0-9]*-[0-9]*-[0-9]*-[0-9]*/ /.idea .vscode .env +.venv **/.DS_Store **/.python-version .coverage @@ -29,4 +30,4 @@ Cargo.lock justfile site -__pycache__ \ No newline at end of file +__pycache__ diff --git a/CHANGELOG.md b/CHANGELOG.md index 922a49f47e..5ec181e6f7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,178 @@ # Changelog +## [rust-v0.17.0](https://github.com/delta-io/delta-rs/tree/rust-v0.17.0) (2024-02-06) + +:warning: The release of 0.17.0 **removes** the legacy dynamodb lock functionality, AWS users must read these release notes! :warning: + +### File handlers + +The 0.17.0 release moves storage implementations into their own crates, such as +`deltalake-aws`. A consequence of that refactoring is that custom storage and +file scheme handlers must be registered/initialized at runtime. Storage +subcrates conventionally define a `register_handlers` function which performs +that task. Users may see errors such as: +``` +thread 'main' panicked at /home/ubuntu/.cargo/registry/src/index.crates.io-6f17d22bba15001f/deltalake-core-0.17.0/src/table/builder.rs:189:48: +The specified table_uri is not valid: InvalidTableLocation("Unknown scheme: s3") +``` + +* Users of the meta-crate (`deltalake`) can call the storage crate via: `deltalake::aws::register_handlers(None);` at the entrypoint for their code. +* Users who adopt `core` and storage crates independently (e.g. `deltalake-aws`) can register via `deltalake_aws::register_handlers(None);`. + +The AWS, Azure, and GCP crates must all have their custom file schemes registered in this fashion. + + +### dynamodblock to S3DynamoDbLogStore + +The locking mechanism is fundamentally different between `deltalake` v0.16.x and v0.17.0, starting with this release the `deltalake` and `deltalake-aws` crates this library now relies on the same [protocol for concurrent writes on AWS](https://docs.delta.io/latest/delta-storage.html#setup-configuration-s3-multi-cluster) as the Delta Lake/Spark implementation. + +Fundamentally the DynamoDB table structure changes, [which is documented here](https://docs.delta.io/latest/delta-storage.html#setup-configuration-s3-multi-cluster). The configuration of a Rust process should continue to use the `AWS_S3_LOCKING_PROVIDER` environment value of `dynamodb`. The new table must be specified with the `DELTA_DYNAMO_TABLE_NAME` environment or configuration variable, and that should name the _new_ `S3DynamoDbLogStore` compatible DynamoDB table. + +Because locking is required to ensure safe cconsistent writes, **there is no iterative migration**, 0.16 and 0.17 writers **cannot** safely coexist. The following steps should be taken when upgrading: + +1. Stop all 0.16.x writers +2. Ensure writes are completed, and lock table is empty. +3. Deploy 0.17.0 writers + + + +[Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.16.5...rust-v0.17.0) + +**Implemented enhancements:** + +- Expose the ability to compile DataFusion with SIMD [\#2118](https://github.com/delta-io/delta-rs/issues/2118) +- Updating Table log retention configuration with `write_deltalake` silently changes nothing [\#2108](https://github.com/delta-io/delta-rs/issues/2108) +- ALTER table, ALTER Column, Add/Modify Comment, Add/remove/rename partitions, Set Tags, Set location, Set TBLProperties [\#2088](https://github.com/delta-io/delta-rs/issues/2088) +- Docs: Update docs for check constraints [\#2063](https://github.com/delta-io/delta-rs/issues/2063) +- Don't `ensure_table_uri` when creating a table `with_log_store` [\#2036](https://github.com/delta-io/delta-rs/issues/2036) +- Exposing custom\_metadata in merge operation [\#2031](https://github.com/delta-io/delta-rs/issues/2031) +- Support custom table properties via TableAlterer and write/merge [\#2022](https://github.com/delta-io/delta-rs/issues/2022) +- Remove parquet2 crate support [\#2004](https://github.com/delta-io/delta-rs/issues/2004) +- Merge operation that only touches necessary partitions [\#1991](https://github.com/delta-io/delta-rs/issues/1991) +- store userMetadata on write operations [\#1990](https://github.com/delta-io/delta-rs/issues/1990) +- Create Dask integration page [\#1956](https://github.com/delta-io/delta-rs/issues/1956) +- Merge: Filtering on partitions [\#1918](https://github.com/delta-io/delta-rs/issues/1918) +- Rethink the load\_version and load\_with\_datetime interfaces [\#1910](https://github.com/delta-io/delta-rs/issues/1910) +- docs: Delta Lake + Arrow Integration [\#1908](https://github.com/delta-io/delta-rs/issues/1908) +- docs: Delta Lake + Polars integration [\#1906](https://github.com/delta-io/delta-rs/issues/1906) +- Rethink decision to expose the public interface in namespaces [\#1900](https://github.com/delta-io/delta-rs/issues/1900) +- Add documentation on how to build and run documentation locally [\#1893](https://github.com/delta-io/delta-rs/issues/1893) +- Add API to create an empty Delta Lake table [\#1892](https://github.com/delta-io/delta-rs/issues/1892) +- Implementing CHECK constraints [\#1881](https://github.com/delta-io/delta-rs/issues/1881) +- Check Invariants are respecting table features for write paths [\#1880](https://github.com/delta-io/delta-rs/issues/1880) +- Organize docs with single lefthand sidebar [\#1873](https://github.com/delta-io/delta-rs/issues/1873) +- Make sure invariants are handled properly throughout the codebase [\#1870](https://github.com/delta-io/delta-rs/issues/1870) +- Unable to use deltalake `Schema` in `write_deltalake` [\#1862](https://github.com/delta-io/delta-rs/issues/1862) +- Add a Rust-backed engine for write\_deltalake [\#1861](https://github.com/delta-io/delta-rs/issues/1861) +- Run doctest in CI for Python API examples [\#1783](https://github.com/delta-io/delta-rs/issues/1783) +- \[RFC\] Use arrow for checkpoint reading and state handling [\#1776](https://github.com/delta-io/delta-rs/issues/1776) +- Expose Python exceptions in public module [\#1771](https://github.com/delta-io/delta-rs/issues/1771) +- Expose cleanup\_metadata or create\_checkpoint\_from\_table\_uri\_and\_cleanup to the Python API [\#1768](https://github.com/delta-io/delta-rs/issues/1768) +- Expose convert\_to\_delta to Python API [\#1767](https://github.com/delta-io/delta-rs/issues/1767) +- Add high-level checking for append-only tables [\#1759](https://github.com/delta-io/delta-rs/issues/1759) + +**Fixed bugs:** + +- Row order no longer preserved after merge operation [\#2165](https://github.com/delta-io/delta-rs/issues/2165) +- Error when reading delta table with IDENTITY column [\#2152](https://github.com/delta-io/delta-rs/issues/2152) +- Merge on IS NULL condition doesn't work for empty table [\#2148](https://github.com/delta-io/delta-rs/issues/2148) +- JsonWriter converts structured parsing error into plain string [\#2143](https://github.com/delta-io/delta-rs/issues/2143) +- Pandas import error when merging tables [\#2112](https://github.com/delta-io/delta-rs/issues/2112) +- test\_repair\_on\_update broken in main [\#2109](https://github.com/delta-io/delta-rs/issues/2109) +- `WriteBuilder::with_input_execution_plan` does not apply the schema to the log's metadata fields [\#2105](https://github.com/delta-io/delta-rs/issues/2105) +- MERGE logical plan vs execution plan schema mismatch [\#2104](https://github.com/delta-io/delta-rs/issues/2104) +- Partitions not pushed down [\#2090](https://github.com/delta-io/delta-rs/issues/2090) +- Cant create empty table with write\_deltalake [\#2086](https://github.com/delta-io/delta-rs/issues/2086) +- Unexpected high costs on Google Cloud Storage [\#2085](https://github.com/delta-io/delta-rs/issues/2085) +- Unable to read s3 table: `Unknown scheme: s3` [\#2065](https://github.com/delta-io/delta-rs/issues/2065) +- write\_deltalake not respecting writer\_properties [\#2064](https://github.com/delta-io/delta-rs/issues/2064) +- Unable to read/write tables with the "gs" schema in the table\_uri in 0.15.1 [\#2060](https://github.com/delta-io/delta-rs/issues/2060) +- LockClient requiered error for S3 backend in 0.15.1 python [\#2057](https://github.com/delta-io/delta-rs/issues/2057) +- Error while writing Pandas DataFrame to Delta Lake \(S3\) [\#2051](https://github.com/delta-io/delta-rs/issues/2051) +- Error with dynamo locking provider on 0.15 [\#2034](https://github.com/delta-io/delta-rs/issues/2034) +- Conda version 0.15.0 is missing files [\#2021](https://github.com/delta-io/delta-rs/issues/2021) +- Rust panicking through Python library when a delete predicate uses a nullable field [\#2019](https://github.com/delta-io/delta-rs/issues/2019) +- No snapshot or version 0 found, perhaps /Users/watsy0007/resources/test\_table/ is an empty dir? [\#2016](https://github.com/delta-io/delta-rs/issues/2016) +- Generic DeltaTable error: type\_coercion in Struct column in merge operation [\#1998](https://github.com/delta-io/delta-rs/issues/1998) +- Constraint expr not formatted during commit action [\#1971](https://github.com/delta-io/delta-rs/issues/1971) +- .load\_with\_datetime\(\) is incorrectly rounding to nearest second [\#1967](https://github.com/delta-io/delta-rs/issues/1967) +- vacuuming log files [\#1965](https://github.com/delta-io/delta-rs/issues/1965) +- Unable to merge uppercase column names [\#1960](https://github.com/delta-io/delta-rs/issues/1960) +- Schema error: Invalid data type for Delta Lake: Null [\#1946](https://github.com/delta-io/delta-rs/issues/1946) +- Python v0.14 wheel files not up to date [\#1945](https://github.com/delta-io/delta-rs/issues/1945) +- python Release 0.14 is missing Windows wheels [\#1942](https://github.com/delta-io/delta-rs/issues/1942) +- CI integration test fails randomly: test\_restore\_by\_datetime [\#1925](https://github.com/delta-io/delta-rs/issues/1925) +- Merge data freezes indefenetely [\#1920](https://github.com/delta-io/delta-rs/issues/1920) +- Load DeltaTable from non-existing folder causing empty folder creation [\#1916](https://github.com/delta-io/delta-rs/issues/1916) +- Reoptimizes merge bins with only 1 file, even though they have no effect. [\#1901](https://github.com/delta-io/delta-rs/issues/1901) +- The Python Docs link in README.MD points to old docs [\#1898](https://github.com/delta-io/delta-rs/issues/1898) +- optimize.compact\(\) fails with bad schema after updating to pyarrow 8.0 [\#1889](https://github.com/delta-io/delta-rs/issues/1889) +- Python build is broken on main [\#1856](https://github.com/delta-io/delta-rs/issues/1856) +- Checkpoint error with Azure Synapse [\#1847](https://github.com/delta-io/delta-rs/issues/1847) +- merge very slow compared to delete + append on larger dataset [\#1846](https://github.com/delta-io/delta-rs/issues/1846) +- get\_add\_actions fails with deltalake 0.13 [\#1835](https://github.com/delta-io/delta-rs/issues/1835) +- Handle PyArrow CVE-2023-47248 [\#1834](https://github.com/delta-io/delta-rs/issues/1834) +- Delta-rs writer hangs with to many file handles open \(Azure\) [\#1832](https://github.com/delta-io/delta-rs/issues/1832) +- Encountering NotATable\("No snapshot or version 0 found, perhaps xxx is an empty dir?"\) [\#1831](https://github.com/delta-io/delta-rs/issues/1831) +- write\_deltalake is not creating checkpoints [\#1815](https://github.com/delta-io/delta-rs/issues/1815) +- Problem writing tables in directory named with char `~` [\#1806](https://github.com/delta-io/delta-rs/issues/1806) +- DeltaTable Merge throws in merging if there are uppercase in Schema. [\#1797](https://github.com/delta-io/delta-rs/issues/1797) +- rust merge error - datafusion panics [\#1790](https://github.com/delta-io/delta-rs/issues/1790) +- expose use\_dictionary=False when writing Delta Table and running optimize [\#1772](https://github.com/delta-io/delta-rs/issues/1772) + +**Closed issues:** + +- Is this print necessary? Can we remove this. [\#2110](https://github.com/delta-io/delta-rs/issues/2110) +- Azure concurrent writes [\#2069](https://github.com/delta-io/delta-rs/issues/2069) +- Fix docs deployment [\#1867](https://github.com/delta-io/delta-rs/issues/1867) +- Add a header in old docs and direct users to new docs [\#1865](https://github.com/delta-io/delta-rs/issues/1865) + +## [rust-v0.16.5](https://github.com/delta-io/delta-rs/tree/rust-v0.16.5) (2023-11-15) + +[Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.16.4...rust-v0.16.5) + +**Implemented enhancements:** + +- When will upgrade object\_store to 0.8? [\#1858](https://github.com/delta-io/delta-rs/issues/1858) +- No Official Help [\#1849](https://github.com/delta-io/delta-rs/issues/1849) +- Auto assign GitHub issues with a "take" message [\#1791](https://github.com/delta-io/delta-rs/issues/1791) + +**Fixed bugs:** + +- cargo clippy fails on core in main [\#1843](https://github.com/delta-io/delta-rs/issues/1843) + +## [rust-v0.16.4](https://github.com/delta-io/delta-rs/tree/rust-v0.16.4) (2023-11-12) + +[Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.16.3...rust-v0.16.4) + +**Implemented enhancements:** + +- Unable to add deltalake git dependency to cargo.toml [\#1821](https://github.com/delta-io/delta-rs/issues/1821) + +## [rust-v0.16.3](https://github.com/delta-io/delta-rs/tree/rust-v0.16.3) (2023-11-08) + +[Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.16.2...rust-v0.16.3) + +**Implemented enhancements:** + +- Docs: add release GitHub action [\#1799](https://github.com/delta-io/delta-rs/issues/1799) +- Use bulk deletes where possible [\#1761](https://github.com/delta-io/delta-rs/issues/1761) + +**Fixed bugs:** + +- Code Owners no longer valid [\#1794](https://github.com/delta-io/delta-rs/issues/1794) +- `MERGE` works incorrectly with partitioned table if the data column order is not same as table column order [\#1787](https://github.com/delta-io/delta-rs/issues/1787) +- errors when using pyarrow dataset as a source [\#1779](https://github.com/delta-io/delta-rs/issues/1779) +- Write to Microsoft OneLake failed. [\#1764](https://github.com/delta-io/delta-rs/issues/1764) + +## [rust-v0.16.2](https://github.com/delta-io/delta-rs/tree/rust-v0.16.2) (2023-10-21) + +[Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.16.1...rust-v0.16.2) + +## [rust-v0.16.1](https://github.com/delta-io/delta-rs/tree/rust-v0.16.1) (2023-10-21) + +[Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.16.0...rust-v0.16.1) + ## [rust-v0.16.0](https://github.com/delta-io/delta-rs/tree/rust-v0.16.0) (2023-09-27) [Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.15.0...rust-v0.16.0) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index ee258a3ce8..4472a3640a 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,6 +1,6 @@ # Contributing to delta-rs -Development on this project is mostly driven by volunteer contributors. We welcome new contributors, including not only those who develop new features, but also those who are able to help with documentation and provide detailed bug reports. +Development on this project is mostly driven by volunteer contributors. We welcome new contributors, including not only those who develop new features, but also those who are able to help with documentation and provide detailed bug reports. Please take note of our [code of conduct](CODE_OF_CONDUCT.md). @@ -31,7 +31,7 @@ python -m pytest tests/test_writer.py -s -k "test_with_deltalake_schema" - Run some Rust code, e.g. run an example ``` cd crates/deltalake -cargo run --examples basic_operations +cargo run --example basic_operations --features="datafusion" ``` ## Run the docs locally diff --git a/Cargo.toml b/Cargo.toml index 1e9f311693..3a99e8ee63 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -7,6 +7,18 @@ members = [ exclude = ["proofs"] resolver = "2" +[workspace.package] +authors = ["Qingping Hou "] +rust-version = "1.72" +keywords = ["deltalake", "delta", "datalake"] +readme = "README.md" +edition = "2021" +description = "Native Delta Lake implementation in Rust" +homepage = "https://github.com/delta-io/delta.rs" +license = "Apache-2.0" +documentation = "https://docs.rs/deltalake" +repository = "https://github.com/delta-io/delta.rs" + [profile.release-with-debug] inherits = "release" debug = true @@ -19,33 +31,36 @@ debug = "line-tables-only" [workspace.dependencies] # arrow -arrow = { version = "48.0.1" } -arrow-arith = { version = "48.0.1" } -arrow-array = { version = "48.0.1" } -arrow-buffer = { version = "48.0.1" } -arrow-cast = { version = "48.0.1" } -arrow-ord = { version = "48.0.1" } -arrow-row = { version = "48.0.1" } -arrow-schema = { version = "48.0.1" } -arrow-select = { version = "48.0.1" } -parquet = { version = "48.0.1" } +arrow = { version = "50" } +arrow-arith = { version = "50" } +arrow-array = { version = "50" } +arrow-buffer = { version = "50" } +arrow-cast = { version = "50" } +arrow-ipc = { version = "50" } +arrow-json = { version = "50" } +arrow-ord = { version = "50" } +arrow-row = { version = "50" } +arrow-schema = { version = "50" } +arrow-select = { version = "50" } +object_store = { version = "=0.9.0" } +parquet = { version = "50" } # datafusion -datafusion = { version = "33.0.0" } -datafusion-expr = { version = "33.0.0" } -datafusion-common = { version = "33.0.0" } -datafusion-proto = { version = "33.0.0" } -datafusion-sql = { version = "33.0.0" } -datafusion-physical-expr = { version = "33.0.0" } - +datafusion = { version = "35" } +datafusion-expr = { version = "35" } +datafusion-common = { version = "35" } +datafusion-proto = { version = "35" } +datafusion-sql = { version = "35" } +datafusion-physical-expr = { version = "35" } # serde -serde = { version = "1", features = ["derive"] } +serde = { version = "1.0.194", features = ["derive"] } serde_json = "1" # "stdlib" bytes = { version = "1" } -chrono = { version = "0.4.31", default-features = false, features = ["clock"] } +chrono = { version = "=0.4.34", default-features = false, features = ["clock"] } +tracing = { version = "0.1", features = ["log"] } regex = { version = "1" } thiserror = { version = "1" } url = { version = "2" } diff --git a/README.md b/README.md index 6121110fb0..100d34355d 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,6 @@

- delta-rs logo + delta-rs logo

@@ -77,7 +77,7 @@ write_deltalake("./data/delta", df) dt = DeltaTable("./data/delta") df2 = dt.to_pandas() -assert df == df2 +assert df.equals(df2) ``` The same table can also be loaded using the core Rust crate: @@ -91,8 +91,8 @@ async fn main() -> Result<(), DeltaTableError> { let table = open_table("./data/delta").await?; // show all active files in the table - let files = table.get_files(); - println!("{files}"); + let files: Vec<_> = table.get_file_uris()?.collect(); + println!("{:?}", files); Ok(()) } @@ -130,45 +130,45 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc ### Cloud Integrations -| Storage | Rust | Python | Comment | -| -------------------- | :-----: | :-----: | ----------------------------------- | -| Local | ![done] | ![done] | | -| S3 - AWS | ![done] | ![done] | requires lock for concurrent writes | -| S3 - MinIO | ![done] | ![done] | requires lock for concurrent writes | -| S3 - R2 | ![done] | ![done] | requires lock for concurrent writes | -| Azure Blob | ![done] | ![done] | | -| Azure ADLS Gen2 | ![done] | ![done] | | -| Microsoft OneLake | ![done] | ![done] | | -| Google Cloud Storage | ![done] | ![done] | | +| Storage | Rust | Python | Comment | +| -------------------- | :-----: | :-----: | ---------------------------------------------------------------- | +| Local | ![done] | ![done] | | +| S3 - AWS | ![done] | ![done] | requires lock for concurrent writes | +| S3 - MinIO | ![done] | ![done] | requires lock for concurrent writes | +| S3 - R2 | ![done] | ![done] | No lock required when using `AmazonS3ConfigKey::CopyIfNotExists` | +| Azure Blob | ![done] | ![done] | | +| Azure ADLS Gen2 | ![done] | ![done] | | +| Microsoft OneLake | ![done] | ![done] | | +| Google Cloud Storage | ![done] | ![done] | | ### Supported Operations -| Operation | Rust | Python | Description | -| --------------------- | :----------------------: | :----------------------: | ------------------------------------------- | -| Create | ![done] | ![done] | Create a new table | -| Read | ![done] | ![done] | Read data from a table | -| Vacuum | ![done] | ![done] | Remove unused files and log entries | -| Delete - partitions | | ![done] | Delete a table partition | -| Delete - predicates | ![done] | ![done] | Delete data based on a predicate | -| Optimize - compaction | ![done] | ![done] | Harmonize the size of data file | -| Optimize - Z-order | ![done] | ![done] | Place similar data into the same file | -| Merge | [![semi-done]][merge-rs] | [![semi-done]][merge-py] | Merge two tables (limited to full re-write) | -| FS check | ![done] | ![done] | Remove corrupted files from table | +| Operation | Rust | Python | Description | +| --------------------- | :-----: | :-----: | ------------------------------------------- | +| Create | ![done] | ![done] | Create a new table | +| Read | ![done] | ![done] | Read data from a table | +| Vacuum | ![done] | ![done] | Remove unused files and log entries | +| Delete - partitions | | ![done] | Delete a table partition | +| Delete - predicates | ![done] | ![done] | Delete data based on a predicate | +| Optimize - compaction | ![done] | ![done] | Harmonize the size of data file | +| Optimize - Z-order | ![done] | ![done] | Place similar data into the same file | +| Merge | ![done] | ![done] | Merge a target Delta table with source data | +| FS check | ![done] | ![done] | Remove corrupted files from table | ### Protocol Support Level -| Writer Version | Requirement | Status | -| -------------- | --------------------------------------------- | :------------------: | -| Version 2 | Append Only Tables | ![done] | -| Version 2 | Column Invariants | ![done] | -| Version 3 | Enforce `delta.checkpoint.writeStatsAsJson` | [![open]][writer-rs] | -| Version 3 | Enforce `delta.checkpoint.writeStatsAsStruct` | [![open]][writer-rs] | -| Version 3 | CHECK constraints | [![open]][writer-rs] | -| Version 4 | Change Data Feed | | -| Version 4 | Generated Columns | | -| Version 5 | Column Mapping | | -| Version 6 | Identity Columns | | -| Version 7 | Table Features | | +| Writer Version | Requirement | Status | +| -------------- | --------------------------------------------- | :-------------------------------: | +| Version 2 | Append Only Tables | ![done] | +| Version 2 | Column Invariants | ![done] | +| Version 3 | Enforce `delta.checkpoint.writeStatsAsJson` | [![open]][writer-rs] | +| Version 3 | Enforce `delta.checkpoint.writeStatsAsStruct` | [![open]][writer-rs] | +| Version 3 | CHECK constraints | [![semi-done]][check-constraints] | +| Version 4 | Change Data Feed | | +| Version 4 | Generated Columns | | +| Version 5 | Column Mapping | | +| Version 6 | Identity Columns | | +| Version 7 | Table Features | | | Reader Version | Requirement | Status | | -------------- | ----------------------------------- | ------ | @@ -182,8 +182,7 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc [semi-done]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/ApprovedChangesGrey.svg [done]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/ApprovedChanges.svg [roadmap]: https://github.com/delta-io/delta-rs/issues/1128 -[merge-py]: https://github.com/delta-io/delta-rs/issues/1357 -[merge-rs]: https://github.com/delta-io/delta-rs/issues/850 [writer-rs]: https://github.com/delta-io/delta-rs/issues/851 +[check-constraints]: https://github.com/delta-io/delta-rs/issues/1881 [onelake-rs]: https://github.com/delta-io/delta-rs/issues/1418 [protocol]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md diff --git a/crates/aws/Cargo.toml b/crates/aws/Cargo.toml new file mode 100644 index 0000000000..169686c240 --- /dev/null +++ b/crates/aws/Cargo.toml @@ -0,0 +1,61 @@ +[package] +name = "deltalake-aws" +version = "0.1.0" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +deltalake-core = { version = "0.17.0", path = "../core" } +aws-smithy-runtime-api = { version="1.1.7" } +aws-smithy-runtime = { version="1.1.7", optional = true} +aws-credential-types = { version="1.1.7", features = ["hardcoded-credentials"]} +aws-config = { version = "1.1.6", default-features = false, features = ["behavior-version-latest","rt-tokio", "credentials-process", "sso"] } +aws-sdk-dynamodb = {version = "1.15.0", default-features = false, features = ["behavior-version-latest", "rt-tokio"] } +aws-sdk-sts = {version = "1.1.6", default-features = false, features = ["behavior-version-latest", "rt-tokio"] } +lazy_static = "1" +maplit = "1" + +# workspace dependencies +async-trait = { workspace = true } +bytes = { workspace = true } +futures = { workspace = true } +tracing = { workspace = true } +object_store = { workspace = true, features = ["aws"]} +thiserror = { workspace = true } +tokio = { workspace = true } +regex = { workspace = true } +uuid = { workspace = true, features = ["serde", "v4"] } +url = { workspace = true } +backoff = { version = "0.4", features = [ "tokio" ] } +hyper-tls = { version = "0.5", optional = true } + +[dev-dependencies] +deltalake-core = { path = "../core", features = ["datafusion"] } +chrono = { workspace = true } +serial_test = "3" +deltalake-test = { path = "../test" } +pretty_env_logger = "0.5.0" +rand = "0.8" +serde_json = { workspace = true } + +[features] +default = ["rustls"] +integration_test = [] +native-tls = [ + "aws-config/client-hyper", + "aws-smithy-runtime/connector-hyper-0-14-x", + "hyper-tls" +] +rustls = [ + "aws-config/client-hyper", + "aws-config/rustls", + "aws-sdk-dynamodb/rustls", + "aws-sdk-sts/rustls", +] diff --git a/crates/deltalake-core/tests/common/s3.rs b/crates/aws/helpers.rs similarity index 98% rename from crates/deltalake-core/tests/common/s3.rs rename to crates/aws/helpers.rs index 1ba5d1cd15..1e6d88a340 100644 --- a/crates/deltalake-core/tests/common/s3.rs +++ b/crates/aws/helpers.rs @@ -39,7 +39,7 @@ pub async fn setup_s3_context() -> TestContext { config.insert("AWS_ACCESS_KEY_ID".to_owned(), "deltalake".to_owned()); config.insert("AWS_SECRET_ACCESS_KEY".to_owned(), "weloverust".to_owned()); config.insert("AWS_S3_LOCKING_PROVIDER".to_owned(), "dynamodb".to_owned()); - config.insert("DYNAMO_LOCK_TABLE_NAME".to_owned(), lock_table.clone()); + config.insert(constants::LOCK_TABLE_KEY_NAME.to_owned(), lock_table.clone()); config.insert("AWS_ALLOW_HTTP".to_owned(), "TRUE".to_string()); TestContext { diff --git a/crates/aws/src/errors.rs b/crates/aws/src/errors.rs new file mode 100644 index 0000000000..a79542bfff --- /dev/null +++ b/crates/aws/src/errors.rs @@ -0,0 +1,170 @@ +//! Errors for S3 log store backed by DynamoDb + +use std::num::ParseIntError; + +use aws_credential_types::provider::error::CredentialsError; +use aws_sdk_dynamodb::{ + error::SdkError, + operation::{ + create_table::CreateTableError, get_item::GetItemError, put_item::PutItemError, + query::QueryError, update_item::UpdateItemError, + }, +}; +use aws_smithy_runtime_api::client::result::ServiceError; + +macro_rules! impl_from_service_error { + ($error_type:ty) => { + impl From> for LockClientError + where + R: Send + Sync + std::fmt::Debug + 'static, + { + fn from(err: SdkError<$error_type, R>) -> Self { + match err { + SdkError::ServiceError(e) => e.into(), + _ => LockClientError::GenericDynamoDb { + source: Box::new(err), + }, + } + } + } + + impl From> for LockClientError + where + R: Send + Sync + std::fmt::Debug + 'static, + { + fn from(value: ServiceError<$error_type, R>) -> Self { + value.into_err().into() + } + } + }; +} + +#[derive(thiserror::Error, Debug)] +pub enum DynamoDbConfigError { + /// Billing mode string invalid + #[error("Invalid billing mode : {0}, supported values : ['provided', 'pay_per_request']")] + InvalidBillingMode(String), + + /// Cannot parse max_elapsed_request_time value into u64 + #[error("Cannot parse max elapsed request time into u64: {source}")] + ParseMaxElapsedRequestTime { + // config_value: String, + source: ParseIntError, + }, + /// Cannot initialize DynamoDbConfiguration due to some sort of threading issue + #[error("Cannot initialize dynamodb lock configuration")] + InitializationError, +} + +/// Errors produced by `DynamoDbLockClient` +#[derive(thiserror::Error, Debug)] +pub enum LockClientError { + #[error("Log item has invalid content: '{description}'")] + InconsistentData { description: String }, + + #[error("Lock table '{name}': creation failed: {source}")] + LockTableCreateFailure { + name: String, + source: Box, + }, + + #[error("Log entry for table '{table_path}' and version '{version}' already exists")] + VersionAlreadyExists { table_path: String, version: i64 }, + + #[error("Provisioned table throughput exceeded")] + ProvisionedThroughputExceeded, + + #[error("Lock table not found")] + LockTableNotFound, + + #[error("error in DynamoDb")] + GenericDynamoDb { + source: Box, + }, + #[error("configuration error: {source}")] + Credentials { source: CredentialsError }, + #[error( + "Atomic rename requires a LockClient for S3 backends. \ + Either configure the LockClient, or set AWS_S3_ALLOW_UNSAFE_RENAME=true \ + to opt out of support for concurrent writers." + )] + LockClientRequired, +} + +impl From for LockClientError { + fn from(err: GetItemError) -> Self { + match err { + GetItemError::ProvisionedThroughputExceededException(_) => { + LockClientError::ProvisionedThroughputExceeded + } + GetItemError::RequestLimitExceeded(_) => LockClientError::ProvisionedThroughputExceeded, + GetItemError::ResourceNotFoundException(_) => LockClientError::LockTableNotFound, + _ => LockClientError::GenericDynamoDb { + source: Box::new(err), + }, + } + } +} + +impl From for LockClientError { + fn from(err: QueryError) -> Self { + match err { + QueryError::ProvisionedThroughputExceededException(_) => { + LockClientError::ProvisionedThroughputExceeded + } + QueryError::RequestLimitExceeded(_) => LockClientError::ProvisionedThroughputExceeded, + QueryError::ResourceNotFoundException(_) => LockClientError::LockTableNotFound, + _ => LockClientError::GenericDynamoDb { + source: Box::new(err), + }, + } + } +} + +impl From for LockClientError { + fn from(err: PutItemError) -> Self { + match err { + PutItemError::ConditionalCheckFailedException(_) => { + unreachable!("error must be handled explicitely") + } + PutItemError::ProvisionedThroughputExceededException(_) => { + LockClientError::ProvisionedThroughputExceeded + } + PutItemError::RequestLimitExceeded(_) => LockClientError::ProvisionedThroughputExceeded, + PutItemError::ResourceNotFoundException(_) => LockClientError::LockTableNotFound, + PutItemError::ItemCollectionSizeLimitExceededException(_) => err.into(), + PutItemError::TransactionConflictException(_) => err.into(), + _ => LockClientError::GenericDynamoDb { + source: Box::new(err), + }, + } + } +} + +impl From for LockClientError { + fn from(err: UpdateItemError) -> Self { + match err { + UpdateItemError::ConditionalCheckFailedException(_) => { + unreachable!("condition check failure in update is not an error") + } + UpdateItemError::InternalServerError(_) => err.into(), + UpdateItemError::ProvisionedThroughputExceededException(_) => { + LockClientError::ProvisionedThroughputExceeded + } + UpdateItemError::RequestLimitExceeded(_) => { + LockClientError::ProvisionedThroughputExceeded + } + UpdateItemError::ResourceNotFoundException(_) => LockClientError::LockTableNotFound, + UpdateItemError::ItemCollectionSizeLimitExceededException(_) => err.into(), + UpdateItemError::TransactionConflictException(_) => err.into(), + _ => LockClientError::GenericDynamoDb { + source: Box::new(err), + }, + } + } +} + +impl_from_service_error!(GetItemError); +impl_from_service_error!(PutItemError); +impl_from_service_error!(QueryError); +impl_from_service_error!(UpdateItemError); diff --git a/crates/aws/src/lib.rs b/crates/aws/src/lib.rs new file mode 100644 index 0000000000..026f0e0df9 --- /dev/null +++ b/crates/aws/src/lib.rs @@ -0,0 +1,672 @@ +//! Lock client implementation based on DynamoDb. + +pub mod errors; +pub mod logstore; +#[cfg(feature = "native-tls")] +mod native; +pub mod storage; +use aws_config::SdkConfig; +use aws_sdk_dynamodb::{ + operation::{ + create_table::CreateTableError, get_item::GetItemError, put_item::PutItemError, + query::QueryError, update_item::UpdateItemError, + }, + types::{ + AttributeDefinition, AttributeValue, BillingMode, KeySchemaElement, KeyType, + ScalarAttributeType, + }, + Client, +}; +use lazy_static::lazy_static; +use object_store::aws::AmazonS3ConfigKey; +use regex::Regex; +use std::{ + collections::HashMap, + str::FromStr, + sync::Arc, + time::{Duration, SystemTime}, +}; +use tracing::debug; + +use deltalake_core::logstore::{logstores, LogStore, LogStoreFactory}; +use deltalake_core::storage::{factories, url_prefix_handler, ObjectStoreRef, StorageOptions}; +use deltalake_core::{DeltaResult, Path}; +use url::Url; + +use errors::{DynamoDbConfigError, LockClientError}; +use storage::{S3ObjectStoreFactory, S3StorageOptions}; + +#[derive(Clone, Debug, Default)] +struct S3LogStoreFactory {} + +impl LogStoreFactory for S3LogStoreFactory { + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + let store = url_prefix_handler(store, Path::parse(location.path())?)?; + + if options + .0 + .contains_key(AmazonS3ConfigKey::CopyIfNotExists.as_ref()) + { + debug!("S3LogStoreFactory has been asked to create a LogStore where the underlying store has copy-if-not-exists enabled - no locking provider required"); + return Ok(deltalake_core::logstore::default_logstore( + store, location, options, + )); + } + + let s3_options = S3StorageOptions::from_map(&options.0)?; + + if s3_options.locking_provider.as_deref() != Some("dynamodb") { + debug!("S3LogStoreFactory has been asked to create a LogStore without the dynamodb locking provider"); + return Ok(deltalake_core::logstore::default_logstore( + store, location, options, + )); + } + + Ok(Arc::new(logstore::S3DynamoDbLogStore::try_new( + location.clone(), + options.clone(), + &s3_options, + store, + )?)) + } +} + +/// Register an [ObjectStoreFactory] for common S3 [Url] schemes +pub fn register_handlers(_additional_prefixes: Option) { + let object_stores = Arc::new(S3ObjectStoreFactory::default()); + let log_stores = Arc::new(S3LogStoreFactory::default()); + for scheme in ["s3", "s3a"].iter() { + let url = Url::parse(&format!("{}://", scheme)).unwrap(); + factories().insert(url.clone(), object_stores.clone()); + logstores().insert(url.clone(), log_stores.clone()); + } +} + +/// Representation of a log entry stored in DynamoDb +/// dynamo db item consists of: +/// - table_path: String - tracked in the log store implementation +/// - file_name: String - commit version.json (part of primary key), stored as i64 in this struct +/// - temp_path: String - name of temporary file containing commit info +/// - complete: bool - operation completed, i.e. atomic rename from `tempPath` to `fileName` succeeded +/// - expire_time: `Option` - epoch seconds at which this external commit entry is safe to be deleted +#[derive(Debug, PartialEq)] +pub struct CommitEntry { + /// Commit version, stored as file name (e.g., 00000N.json) in dynamodb (relative to `_delta_log/` + pub version: i64, + /// Path to temp file for this commit, relative to the `_delta_log + pub temp_path: Path, + /// true if delta json file is successfully copied to its destination location, else false + pub complete: bool, + /// If complete = true, epoch seconds at which this external commit entry is safe to be deleted + pub expire_time: Option, +} + +impl CommitEntry { + /// Create a new log entry for the given version. + /// Initial log entry state is incomplete. + pub fn new(version: i64, temp_path: Path) -> CommitEntry { + Self { + version, + temp_path, + complete: false, + expire_time: None, + } + } +} + +/// Lock client backed by DynamoDb. +pub struct DynamoDbLockClient { + /// DynamoDb client + dynamodb_client: Client, + /// configuration of the + config: DynamoDbConfig, +} + +impl std::fmt::Debug for DynamoDbLockClient { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(f, "DynamoDbLockClient(config: {:?})", self.config) + } +} + +impl DynamoDbLockClient { + /// Creates a new DynamoDbLockClient from the supplied storage options. + pub fn try_new( + sdk_config: &SdkConfig, + lock_table_name: Option, + billing_mode: Option, + max_elapsed_request_time: Option, + ) -> Result { + let dynamodb_client = aws_sdk_dynamodb::Client::new(sdk_config); + + let lock_table_name = lock_table_name + .or_else(|| std::env::var(constants::LOCK_TABLE_KEY_NAME).ok()) + .unwrap_or(constants::DEFAULT_LOCK_TABLE_NAME.to_owned()); + + let billing_mode = if let Some(bm) = billing_mode + .or_else(|| std::env::var(constants::BILLING_MODE_KEY_NAME).ok()) + .as_ref() + { + BillingMode::try_parse(bm.to_ascii_uppercase().as_str()) + .map_err(|_| DynamoDbConfigError::InvalidBillingMode(String::default()))? + } else { + BillingMode::PayPerRequest + }; + + let max_elapsed_request_time = max_elapsed_request_time + .or_else(|| std::env::var(constants::MAX_ELAPSED_REQUEST_TIME_KEY_NAME).ok()) + .map_or_else( + || Ok(Duration::from_secs(60)), + |secs| u64::from_str(&secs).map(Duration::from_secs), + ) + .map_err(|err| DynamoDbConfigError::ParseMaxElapsedRequestTime { source: err })?; + + let config = DynamoDbConfig { + billing_mode, + lock_table_name, + max_elapsed_request_time, + sdk_config: sdk_config.clone(), + }; + Ok(Self { + dynamodb_client, + config, + }) + } + + /// Create the lock table where DynamoDb stores the commit information for all delta tables. + /// + /// Transparently handles the case where that table already exists, so it's safe to call. + /// After `create_table` operation is executed, the table state in DynamoDb is `creating`, and + /// it's not immediately useable. This method does not wait for the table state to become + /// `active`, so transient failures might occurr when immediately using the lock client. + pub async fn try_create_lock_table(&self) -> Result { + let attribute_definitions = vec![ + AttributeDefinition::builder() + .attribute_name(constants::ATTR_TABLE_PATH) + .attribute_type(ScalarAttributeType::S) + .build() + .unwrap(), + AttributeDefinition::builder() + .attribute_name(constants::ATTR_FILE_NAME) + .attribute_type(ScalarAttributeType::S) + .build() + .unwrap(), + ]; + let request = self + .dynamodb_client + .create_table() + .set_attribute_definitions(Some(attribute_definitions)) + .set_key_schema(Some(vec![ + KeySchemaElement::builder() + .attribute_name(constants::ATTR_TABLE_PATH.to_owned()) + .key_type(KeyType::Hash) + .build() + .unwrap(), + KeySchemaElement::builder() + .attribute_name(constants::ATTR_FILE_NAME.to_owned()) + .key_type(KeyType::Range) + .build() + .unwrap(), + ])) + .billing_mode(self.config.billing_mode.clone()) + .table_name(&self.config.lock_table_name) + .send(); + match request.await { + Ok(_) => Ok(CreateLockTableResult::TableCreated), + Err(sdk_err) => match sdk_err.as_service_error() { + Some(CreateTableError::ResourceInUseException(_)) => { + Ok(CreateLockTableResult::TableAlreadyExists) + } + Some(_) => Err(LockClientError::LockTableCreateFailure { + name: self.config.lock_table_name.clone(), + source: Box::new(sdk_err.into_service_error()), + }), + _ => Err(LockClientError::GenericDynamoDb { + source: Box::new(sdk_err), + }), + }, + } + } + + /// Get the name of the lock table for transactional commits used by the DynamoDb lock client. + pub fn get_lock_table_name(&self) -> String { + self.config.lock_table_name.clone() + } + + pub fn get_dynamodb_config(&self) -> &DynamoDbConfig { + &self.config + } + + fn get_primary_key(&self, version: i64, table_path: &str) -> HashMap { + maplit::hashmap! { + constants::ATTR_TABLE_PATH.to_owned() => string_attr(table_path), + constants::ATTR_FILE_NAME.to_owned() => string_attr(format!("{:020}.json", version)), + } + } + + /// Read a log entry from DynamoDb. + pub async fn get_commit_entry( + &self, + table_path: &str, + version: i64, + ) -> Result, LockClientError> { + let item = self + .retry(|| async { + match self + .dynamodb_client + .get_item() + .consistent_read(true) + .table_name(&self.config.lock_table_name) + .set_key(Some(self.get_primary_key(version, table_path))) + .send() + .await + { + Ok(x) => Ok(x), + Err(sdk_err) => match sdk_err.as_service_error() { + Some(GetItemError::ProvisionedThroughputExceededException(_)) => { + Err(backoff::Error::transient( + LockClientError::ProvisionedThroughputExceeded, + )) + } + _ => Err(backoff::Error::permanent(sdk_err.into())), + }, + } + }) + .await?; + item.item.as_ref().map(CommitEntry::try_from).transpose() + } + + /// write new entry to to DynamoDb lock table. + pub async fn put_commit_entry( + &self, + table_path: &str, + entry: &CommitEntry, + ) -> Result<(), LockClientError> { + self.retry(|| async { + let item = create_value_map(entry, table_path); + match self + .dynamodb_client + .put_item() + .condition_expression(constants::CONDITION_EXPR_CREATE.as_str()) + .table_name(self.get_lock_table_name()) + .set_item(Some(item)) + .send() + .await + { + Ok(_) => Ok(()), + Err(err) => match err.as_service_error() { + Some(PutItemError::ProvisionedThroughputExceededException(_)) => Err( + backoff::Error::transient(LockClientError::ProvisionedThroughputExceeded), + ), + Some(PutItemError::ConditionalCheckFailedException(_)) => Err( + backoff::Error::permanent(LockClientError::VersionAlreadyExists { + table_path: table_path.to_owned(), + version: entry.version, + }), + ), + Some(PutItemError::ResourceNotFoundException(_)) => Err( + backoff::Error::permanent(LockClientError::LockTableNotFound), + ), + _ => Err(backoff::Error::permanent(err.into())), + }, + } + }) + .await + } + + /// Get the latest entry (entry with highest version). + pub async fn get_latest_entry( + &self, + table_path: &str, + ) -> Result, LockClientError> { + Ok(self + .get_latest_entries(table_path, 1) + .await? + .into_iter() + .next()) + } + + /// Find the latest entry in the lock table for the delta table on the specified `table_path`. + pub async fn get_latest_entries( + &self, + table_path: &str, + limit: i64, + ) -> Result, LockClientError> { + let query_result = self + .retry(|| async { + match self + .dynamodb_client + .query() + .table_name(self.get_lock_table_name()) + .consistent_read(true) + .limit(limit.try_into().unwrap_or(i32::MAX)) + .scan_index_forward(false) + .key_condition_expression(format!("{} = :tn", constants::ATTR_TABLE_PATH)) + .set_expression_attribute_values(Some( + maplit::hashmap!(":tn".into() => string_attr(table_path)), + )) + .send() + .await + { + Ok(result) => Ok(result), + Err(sdk_err) => match sdk_err.as_service_error() { + Some(QueryError::ProvisionedThroughputExceededException(_)) => { + Err(backoff::Error::transient( + LockClientError::ProvisionedThroughputExceeded, + )) + } + _ => Err(backoff::Error::permanent(sdk_err.into())), + }, + } + }) + .await?; + + query_result + .items + .unwrap() + .iter() + .map(CommitEntry::try_from) + .collect() + } + + /// Update existing log entry + pub async fn update_commit_entry( + &self, + version: i64, + table_path: &str, + ) -> Result { + let seconds_since_epoch = (SystemTime::now() + + constants::DEFAULT_COMMIT_ENTRY_EXPIRATION_DELAY) + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs(); + self.retry(|| async { + match self + .dynamodb_client + .update_item() + .table_name(self.get_lock_table_name()) + .set_key(Some(self.get_primary_key(version, table_path))) + .update_expression("SET complete = :c, expireTime = :e".to_owned()) + .set_expression_attribute_values(Some(maplit::hashmap! { + ":c".to_owned() => string_attr("true"), + ":e".to_owned() => num_attr(seconds_since_epoch), + ":f".into() => string_attr("false"), + })) + .condition_expression(constants::CONDITION_UPDATE_INCOMPLETE) + .send() + .await + { + Ok(_) => Ok(UpdateLogEntryResult::UpdatePerformed), + Err(err) => match err.as_service_error() { + Some(UpdateItemError::ProvisionedThroughputExceededException(_)) => Err( + backoff::Error::transient(LockClientError::ProvisionedThroughputExceeded), + ), + Some(UpdateItemError::ConditionalCheckFailedException(_)) => { + Ok(UpdateLogEntryResult::AlreadyCompleted) + } + _ => Err(backoff::Error::permanent(err.into())), + }, + } + }) + .await + } + + async fn retry(&self, operation: Fn) -> Result + where + Fn: FnMut() -> Fut, + Fut: std::future::Future>>, + { + let backoff = backoff::ExponentialBackoffBuilder::new() + .with_multiplier(2.) + .with_max_interval(Duration::from_secs(15)) + .with_max_elapsed_time(Some(self.config.max_elapsed_request_time)) + .build(); + backoff::future::retry(backoff, operation).await + } +} + +#[derive(Debug, PartialEq)] +pub enum UpdateLogEntryResult { + UpdatePerformed, + AlreadyCompleted, +} + +impl TryFrom<&HashMap> for CommitEntry { + type Error = LockClientError; + + fn try_from(item: &HashMap) -> Result { + let version_str = extract_required_string_field(item, constants::ATTR_FILE_NAME)?; + let version = extract_version_from_filename(version_str).ok_or_else(|| { + LockClientError::InconsistentData { + description: format!( + "invalid log file name: can't extract version number from '{version_str}'" + ), + } + })?; + let temp_path = extract_required_string_field(item, constants::ATTR_TEMP_PATH)?; + let temp_path = + Path::from_iter(DELTA_LOG_PATH.parts().chain(Path::from(temp_path).parts())); + let expire_time: Option = + extract_optional_number_field(item, constants::ATTR_EXPIRE_TIME)? + .map(|s| { + s.parse::() + .map_err(|err| LockClientError::InconsistentData { + description: format!("conversion to number failed, {err}"), + }) + }) + .transpose()? + .map(epoch_to_system_time); + Ok(Self { + version, + temp_path, + complete: extract_required_string_field(item, constants::ATTR_COMPLETE)? == "true", + expire_time, + }) + } +} + +fn system_time_to_epoch(t: &SystemTime) -> u64 { + t.duration_since(SystemTime::UNIX_EPOCH).unwrap().as_secs() +} + +fn epoch_to_system_time(s: u64) -> SystemTime { + SystemTime::UNIX_EPOCH + Duration::from_secs(s) +} + +fn create_value_map( + commit_entry: &CommitEntry, + table_path: &str, +) -> HashMap { + // cut off `_delta_log` part: temp_path in DynamoDb is relative to `_delta_log` not table root. + let temp_path = Path::from_iter(commit_entry.temp_path.parts().skip(1)); + let mut value_map = maplit::hashmap! { + constants::ATTR_TABLE_PATH.to_owned() => string_attr(table_path), + constants::ATTR_FILE_NAME.to_owned() => string_attr(format!("{:020}.json", commit_entry.version)), + constants::ATTR_TEMP_PATH.to_owned() => string_attr(temp_path), + constants::ATTR_COMPLETE.to_owned() => string_attr(if commit_entry.complete { "true" } else { "false" }), + }; + commit_entry.expire_time.as_ref().map(|t| { + value_map.insert( + constants::ATTR_EXPIRE_TIME.to_owned(), + num_attr(system_time_to_epoch(t)), + ) + }); + value_map +} + +#[derive(Debug)] +pub struct DynamoDbConfig { + pub billing_mode: BillingMode, + pub lock_table_name: String, + pub max_elapsed_request_time: Duration, + pub sdk_config: SdkConfig, +} + +impl Eq for DynamoDbConfig {} +impl PartialEq for DynamoDbConfig { + fn eq(&self, other: &Self) -> bool { + self.billing_mode == other.billing_mode + && self.lock_table_name == other.lock_table_name + && self.max_elapsed_request_time == other.max_elapsed_request_time + && self.sdk_config.endpoint_url() == other.sdk_config.endpoint_url() + && self.sdk_config.region() == other.sdk_config.region() + } +} + +/// Represents the possible, positive outcomes of calling `DynamoDbClient::try_create_lock_table()` +#[derive(Debug, PartialEq)] +pub enum CreateLockTableResult { + /// Table created successfully. + TableCreated, + /// Table was not created because it already exists. + /// Does not imply that the table has the correct schema. + TableAlreadyExists, +} + +pub mod constants { + use std::time::Duration; + + use lazy_static::lazy_static; + + pub const DEFAULT_LOCK_TABLE_NAME: &str = "delta_log"; + pub const LOCK_TABLE_KEY_NAME: &str = "DELTA_DYNAMO_TABLE_NAME"; + pub const BILLING_MODE_KEY_NAME: &str = "DELTA_DYNAMO_BILLING_MODE"; + pub const MAX_ELAPSED_REQUEST_TIME_KEY_NAME: &str = "DELTA_DYNAMO_MAX_ELAPSED_REQUEST_TIME"; + + pub const ATTR_TABLE_PATH: &str = "tablePath"; + pub const ATTR_FILE_NAME: &str = "fileName"; + pub const ATTR_TEMP_PATH: &str = "tempPath"; + pub const ATTR_COMPLETE: &str = "complete"; + pub const ATTR_EXPIRE_TIME: &str = "expireTime"; + + pub const STRING_TYPE: &str = "S"; + + pub const KEY_TYPE_HASH: &str = "HASH"; + pub const KEY_TYPE_RANGE: &str = "RANGE"; + + lazy_static! { + pub static ref CONDITION_EXPR_CREATE: String = format!( + "attribute_not_exists({ATTR_TABLE_PATH}) and attribute_not_exists({ATTR_FILE_NAME})" + ); + } + + pub const CONDITION_UPDATE_INCOMPLETE: &str = "complete = :f"; + + pub const DEFAULT_COMMIT_ENTRY_EXPIRATION_DELAY: Duration = Duration::from_secs(86_400); +} + +/// Extract a field from an item's attribute value map, producing a descriptive error +/// of the various failure cases. +fn extract_required_string_field<'a>( + fields: &'a HashMap, + field_name: &str, +) -> Result<&'a str, LockClientError> { + fields + .get(field_name) + .ok_or_else(|| LockClientError::InconsistentData { + description: format!("mandatory string field '{field_name}' missing"), + })? + .as_s() + .map_err(|v| LockClientError::InconsistentData { + description: format!( + "mandatory string field '{field_name}' exists, but is not a string: {:#?}", + v, + ), + }) + .map(|s| s.as_str()) +} + +/// Extract an optional String field from an item's attribute value map. +/// This call fails if the field exists, but is not of type string. +fn extract_optional_number_field<'a>( + fields: &'a HashMap, + field_name: &str, +) -> Result, LockClientError> { + fields + .get(field_name) + .map(|attr| { + attr.as_n().map_err(|_| LockClientError::InconsistentData { + description: format!( + "field with name '{field_name}' exists, but is not of type number" + ), + }) + }) + .transpose() +} + +fn string_attr(s: T) -> AttributeValue { + AttributeValue::S(s.to_string()) +} + +fn num_attr(n: T) -> AttributeValue { + AttributeValue::N(n.to_string()) +} + +lazy_static! { + static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); + static ref DELTA_LOG_REGEX: Regex = Regex::new(r"(\d{20})\.(json|checkpoint).*$").unwrap(); +} + +/// Extract version from a file name in the delta log +fn extract_version_from_filename(name: &str) -> Option { + DELTA_LOG_REGEX + .captures(name) + .map(|captures| captures.get(1).unwrap().as_str().parse().unwrap()) +} + +#[cfg(test)] +mod tests { + use super::*; + use object_store::memory::InMemory; + use serial_test::serial; + + fn commit_entry_roundtrip(c: &CommitEntry) -> Result<(), LockClientError> { + let item_data: HashMap = create_value_map(c, "some_table"); + let c_parsed = CommitEntry::try_from(&item_data)?; + assert_eq!(c, &c_parsed); + Ok(()) + } + + #[test] + fn commit_entry_roundtrip_test() -> Result<(), LockClientError> { + let system_time = SystemTime::UNIX_EPOCH + + Duration::from_secs( + SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs(), + ); + commit_entry_roundtrip(&CommitEntry { + version: 0, + temp_path: Path::from("_delta_log/tmp/0_abc.json"), + complete: true, + expire_time: Some(system_time), + })?; + commit_entry_roundtrip(&CommitEntry { + version: 139, + temp_path: Path::from("_delta_log/tmp/0_abc.json"), + complete: false, + expire_time: None, + })?; + Ok(()) + } + + /// In cases where there is no dynamodb specified locking provider, this should get a default + /// logstore + #[test] + #[serial] + fn test_logstore_factory_default() { + let factory = S3LogStoreFactory::default(); + let store = InMemory::new(); + let url = Url::parse("s3://test-bucket").unwrap(); + std::env::remove_var(storage::s3_constants::AWS_S3_LOCKING_PROVIDER); + let logstore = factory + .with_options(Arc::new(store), &url, &StorageOptions::from(HashMap::new())) + .unwrap(); + assert_eq!(logstore.name(), "DefaultLogStore"); + } +} diff --git a/crates/deltalake-core/src/logstore/s3/mod.rs b/crates/aws/src/logstore.rs similarity index 70% rename from crates/deltalake-core/src/logstore/s3/mod.rs rename to crates/aws/src/logstore.rs index 9e7883c7b2..a9a98dbe18 100644 --- a/crates/deltalake-core/src/logstore/s3/mod.rs +++ b/crates/aws/src/logstore.rs @@ -3,22 +3,22 @@ //! when the underlying object storage does not support atomic `put_if_absent` //! or `rename_if_absent` operations, as is the case for S3. -use deltalake_aws::errors::LockClientError; -use deltalake_aws::{constants, CommitEntry, DynamoDbLockClient, UpdateLogEntryResult}; +use crate::errors::LockClientError; +use crate::storage::S3StorageOptions; +use crate::{constants, CommitEntry, DynamoDbLockClient, UpdateLogEntryResult}; use bytes::Bytes; -use object_store::path::Path; -use object_store::Error as ObjectStoreError; +use deltalake_core::{ObjectStoreError, Path}; +use tracing::{debug, error, warn}; use url::Url; -use crate::{ +use deltalake_core::logstore::*; +use deltalake_core::{ operations::transaction::TransactionError, - storage::{config::StorageOptions, s3::S3StorageOptions, ObjectStoreRef}, + storage::{ObjectStoreRef, StorageOptions}, DeltaResult, DeltaTableError, }; -use super::{LogStore, LogStoreConfig}; - const STORE_NAME: &str = "DeltaS3ObjectStore"; const MAX_REPAIR_RETRIES: i64 = 3; @@ -30,6 +30,12 @@ pub struct S3DynamoDbLogStore { table_path: String, } +impl std::fmt::Debug for S3DynamoDbLogStore { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(f, "S3DynamoDbLogStore({})", self.table_path) + } +} + impl S3DynamoDbLogStore { /// Create log store pub fn try_new( @@ -39,18 +45,27 @@ impl S3DynamoDbLogStore { object_store: ObjectStoreRef, ) -> DeltaResult { let lock_client = DynamoDbLockClient::try_new( - s3_options.extra_opts.get(constants::LOCK_TABLE_KEY_NAME), - s3_options.extra_opts.get(constants::BILLING_MODE_KEY_NAME), - s3_options.region.clone(), - s3_options.use_web_identity, + &s3_options.sdk_config, + s3_options + .extra_opts + .get(constants::LOCK_TABLE_KEY_NAME) + .cloned(), + s3_options + .extra_opts + .get(constants::BILLING_MODE_KEY_NAME) + .cloned(), + s3_options + .extra_opts + .get(constants::MAX_ELAPSED_REQUEST_TIME_KEY_NAME) + .cloned(), ) .map_err(|err| DeltaTableError::ObjectStore { - source: object_store::Error::Generic { + source: ObjectStoreError::Generic { store: STORE_NAME, - source: err.into(), + source: Box::new(err), }, })?; - let table_path = super::to_uri(&location, &Path::from("")); + let table_path = to_uri(&location, &Path::from("")); Ok(Self { storage: object_store, lock_client, @@ -73,22 +88,22 @@ impl S3DynamoDbLogStore { return Ok(RepairLogEntryResult::AlreadyCompleted); } for retry in 0..=MAX_REPAIR_RETRIES { - match super::write_commit_entry(self.storage.as_ref(), entry.version, &entry.temp_path) - .await - { + match write_commit_entry(&self.storage, entry.version, &entry.temp_path).await { Ok(()) => { + debug!("Successfully committed entry for version {}", entry.version); return self.try_complete_entry(entry, true).await; } // `N.json` has already been moved, complete the entry in DynamoDb just in case Err(TransactionError::ObjectStore { source: ObjectStoreError::NotFound { .. }, }) => { + warn!("It looks like the {}.json has already been moved, we got 404 from ObjectStorage.", entry.version); return self.try_complete_entry(entry, false).await; } Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), - Err(err) => log::debug!( - "retry #{retry} on log entry {entry:?} failed to move commit: '{err}'" - ), + Err(err) => { + debug!("retry #{retry} on log entry {entry:?} failed to move commit: '{err}'") + } } } unreachable!("for loop yields Ok or Err in body when retry = MAX_REPAIR_RETRIES") @@ -100,6 +115,7 @@ impl S3DynamoDbLogStore { entry: &CommitEntry, copy_performed: bool, ) -> Result { + debug!("try_complete_entry for {:?}, {}", entry, copy_performed); for retry in 0..=MAX_REPAIR_RETRIES { match self .lock_client @@ -114,7 +130,7 @@ impl S3DynamoDbLogStore { }) { Ok(x) => return Ok(Self::map_retry_result(x, copy_performed)), Err(err) if retry == MAX_REPAIR_RETRIES => return Err(err), - Err(err) => log::debug!( + Err(err) => error!( "retry #{retry} on log entry {entry:?} failed to update lock db: '{err}'" ), } @@ -141,10 +157,28 @@ impl S3DynamoDbLogStore { #[async_trait::async_trait] impl LogStore for S3DynamoDbLogStore { + fn name(&self) -> String { + "S3DynamoDbLogStore".into() + } + fn root_uri(&self) -> String { self.table_path.clone() } + async fn refresh(&self) -> DeltaResult<()> { + let entry = self + .lock_client + .get_latest_entry(&self.table_path) + .await + .map_err(|err| DeltaTableError::GenericError { + source: Box::new(err), + })?; + if let Some(entry) = entry { + self.repair_entry(&entry).await?; + } + Ok(()) + } + async fn read_commit_entry(&self, version: i64) -> DeltaResult> { let entry = self .lock_client @@ -153,7 +187,7 @@ impl LogStore for S3DynamoDbLogStore { if let Ok(Some(entry)) = entry { self.repair_entry(&entry).await?; } - super::read_commit_entry(self.storage.as_ref(), version).await + read_commit_entry(&self.storage, version).await } /// Tries to commit a prepared commit file. Returns [DeltaTableError::VersionAlreadyExists] @@ -167,26 +201,34 @@ impl LogStore for S3DynamoDbLogStore { tmp_commit: &Path, ) -> Result<(), TransactionError> { let entry = CommitEntry::new(version, tmp_commit.clone()); + debug!("Writing commit entry for {self:?}: {entry:?}"); // create log entry in dynamo db: complete = false, no expireTime self.lock_client .put_commit_entry(&self.table_path, &entry) .await .map_err(|err| match err { LockClientError::VersionAlreadyExists { version, .. } => { + warn!("LockClientError::VersionAlreadyExists({version})"); TransactionError::VersionAlreadyExists(version) } - LockClientError::ProvisionedThroughputExceeded => todo!(), - LockClientError::LockTableNotFound => TransactionError::LogStoreError { - msg: format!( - "lock table '{}' not found", - self.lock_client.get_lock_table_name() - ), - source: Box::new(err), - }, - err => TransactionError::LogStoreError { - msg: "dynamodb client failed to write log entry".to_owned(), - source: Box::new(err), - }, + LockClientError::ProvisionedThroughputExceeded => todo!( + "deltalake-aws does not yet handle DynamoDB provisioned throughput errors" + ), + LockClientError::LockTableNotFound => { + let table_name = self.lock_client.get_lock_table_name(); + error!("Lock table '{table_name}' not found"); + TransactionError::LogStoreError { + msg: format!("lock table '{table_name}' not found"), + source: Box::new(err), + } + } + err => { + error!("dynamodb client failed to write log entry: {err:?}"); + TransactionError::LogStoreError { + msg: "dynamodb client failed to write log entry".to_owned(), + source: Box::new(err), + } + } })?; // `repair_entry` performs the exact steps required to finalize the commit, but contains // retry logic and more robust error handling under the assumption that any other client @@ -198,6 +240,7 @@ impl LogStore for S3DynamoDbLogStore { } async fn get_latest_version(&self, current_version: i64) -> DeltaResult { + debug!("Retrieving latest version of {self:?} at v{current_version}"); let entry = self .lock_client .get_latest_entry(&self.table_path) @@ -210,7 +253,7 @@ impl LogStore for S3DynamoDbLogStore { self.repair_entry(&entry).await?; Ok(entry.version) } else { - super::get_latest_version(self, current_version).await + get_latest_version(self, current_version).await } } @@ -218,15 +261,6 @@ impl LogStore for S3DynamoDbLogStore { self.storage.clone() } - fn to_uri(&self, location: &Path) -> String { - super::to_uri(&self.config.location, location) - } - - #[cfg(feature = "datafusion")] - fn object_store_url(&self) -> datafusion::execution::object_store::ObjectStoreUrl { - super::object_store_url(&self.config.location) - } - fn config(&self) -> &LogStoreConfig { &self.config } diff --git a/crates/aws/src/native.rs b/crates/aws/src/native.rs new file mode 100644 index 0000000000..c647194eb7 --- /dev/null +++ b/crates/aws/src/native.rs @@ -0,0 +1,12 @@ +use aws_sdk_sts::config::SharedHttpClient; +use aws_smithy_runtime::client::http::hyper_014::HyperClientBuilder; + +pub fn use_native_tls_client(allow_http: bool) -> SharedHttpClient { + let mut tls_connector = hyper_tls::HttpsConnector::new(); + if allow_http { + tls_connector.https_only(false); + } + + let client = HyperClientBuilder::new().build(tls_connector); + client +} diff --git a/crates/aws/src/storage.rs b/crates/aws/src/storage.rs new file mode 100644 index 0000000000..ffd7bb6996 --- /dev/null +++ b/crates/aws/src/storage.rs @@ -0,0 +1,747 @@ +//! AWS S3 storage backend. + +use aws_config::{Region, SdkConfig}; +use bytes::Bytes; +use deltalake_core::storage::object_store::{ + aws::AmazonS3ConfigKey, parse_url_opts, GetOptions, GetResult, ListResult, MultipartId, + ObjectMeta, ObjectStore, PutOptions, PutResult, Result as ObjectStoreResult, +}; +use deltalake_core::storage::{str_is_truthy, ObjectStoreFactory, ObjectStoreRef, StorageOptions}; +use deltalake_core::{DeltaResult, ObjectStoreError, Path}; +use futures::stream::BoxStream; +use futures::Future; +use std::collections::HashMap; +use std::fmt::Debug; +use std::ops::Range; +use std::str::FromStr; +use std::sync::Arc; +use std::time::Duration; +use tokio::io::AsyncWrite; +use url::Url; + +use crate::errors::DynamoDbConfigError; +#[cfg(feature = "native-tls")] +use crate::native; + +const STORE_NAME: &str = "DeltaS3ObjectStore"; + +#[derive(Clone, Default, Debug)] +pub struct S3ObjectStoreFactory {} + +impl S3ObjectStoreFactory { + fn with_env_s3(&self, options: &StorageOptions) -> StorageOptions { + let mut options = StorageOptions( + options + .0 + .clone() + .into_iter() + .map(|(k, v)| { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()) { + (config_key.as_ref().to_string(), v) + } else { + (k, v) + } + }) + .collect(), + ); + + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { + if !options.0.contains_key(config_key.as_ref()) { + options + .0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + options + } +} + +impl ObjectStoreFactory for S3ObjectStoreFactory { + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + let options = self.with_env_s3(options); + let (store, prefix) = parse_url_opts( + url, + options.0.iter().filter_map(|(key, value)| { + let s3_key = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; + Some((s3_key, value.clone())) + }), + )?; + + if options + .0 + .contains_key(AmazonS3ConfigKey::CopyIfNotExists.as_ref()) + { + // If the copy-if-not-exists env var is set, we don't need to instantiate a locking client or check for allow-unsafe-rename. + return Ok((Arc::from(store), prefix)); + } + + let options = S3StorageOptions::from_map(&options.0)?; + + let store = S3StorageBackend::try_new( + store.into(), + Some("dynamodb") == options.locking_provider.as_deref() || options.allow_unsafe_rename, + )?; + + Ok((Arc::new(store), prefix)) + } +} + +/// Options used to configure the [S3StorageBackend]. +/// +/// Available options are described in [s3_constants]. +#[derive(Clone, Debug)] +#[allow(missing_docs)] +pub struct S3StorageOptions { + pub virtual_hosted_style_request: bool, + pub locking_provider: Option, + pub s3_pool_idle_timeout: Duration, + pub sts_pool_idle_timeout: Duration, + pub s3_get_internal_server_error_retries: usize, + pub allow_unsafe_rename: bool, + pub extra_opts: HashMap, + pub sdk_config: SdkConfig, +} + +impl Eq for S3StorageOptions {} +impl PartialEq for S3StorageOptions { + fn eq(&self, other: &Self) -> bool { + self.virtual_hosted_style_request == other.virtual_hosted_style_request + && self.locking_provider == other.locking_provider + && self.s3_pool_idle_timeout == other.s3_pool_idle_timeout + && self.sts_pool_idle_timeout == other.sts_pool_idle_timeout + && self.s3_get_internal_server_error_retries + == other.s3_get_internal_server_error_retries + && self.allow_unsafe_rename == other.allow_unsafe_rename + && self.extra_opts == other.extra_opts + && self.sdk_config.endpoint_url() == other.sdk_config.endpoint_url() + && self.sdk_config.region() == other.sdk_config.region() + } +} + +impl S3StorageOptions { + /// Creates an instance of S3StorageOptions from the given HashMap. + pub fn from_map(options: &HashMap) -> DeltaResult { + let extra_opts = options + .iter() + .filter(|(k, _)| !s3_constants::S3_OPTS.contains(&k.as_str())) + .map(|(k, v)| (k.to_owned(), v.to_owned())) + .collect(); + + // Copy web identity values provided in options but not the environment into the environment + // to get picked up by the `from_k8s_env` call in `get_web_identity_provider`. + Self::ensure_env_var(options, s3_constants::AWS_REGION); + Self::ensure_env_var(options, s3_constants::AWS_PROFILE); + Self::ensure_env_var(options, s3_constants::AWS_ACCESS_KEY_ID); + Self::ensure_env_var(options, s3_constants::AWS_SECRET_ACCESS_KEY); + Self::ensure_env_var(options, s3_constants::AWS_SESSION_TOKEN); + Self::ensure_env_var(options, s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE); + Self::ensure_env_var(options, s3_constants::AWS_ROLE_ARN); + Self::ensure_env_var(options, s3_constants::AWS_ROLE_SESSION_NAME); + let s3_pool_idle_timeout = + Self::u64_or_default(options, s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, 15); + let sts_pool_idle_timeout = + Self::u64_or_default(options, s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, 10); + + let s3_get_internal_server_error_retries = Self::u64_or_default( + options, + s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + 10, + ) as usize; + + let virtual_hosted_style_request: bool = + str_option(options, s3_constants::AWS_S3_ADDRESSING_STYLE) + .map(|addressing_style| addressing_style == "virtual") + .unwrap_or(false); + + let allow_unsafe_rename = str_option(options, s3_constants::AWS_S3_ALLOW_UNSAFE_RENAME) + .map(|val| str_is_truthy(&val)) + .unwrap_or(false); + + #[cfg(feature = "native-tls")] + let sdk_config = execute_sdk_future( + aws_config::ConfigLoader::default() + .http_client(native::use_native_tls_client( + str_option(options, s3_constants::AWS_ALLOW_HTTP) + .map(|val| str_is_truthy(&val)) + .unwrap_or(false), + )) + .load(), + )?; + #[cfg(feature = "rustls")] + let sdk_config = execute_sdk_future(aws_config::load_from_env())?; + + let sdk_config = + if let Some(endpoint_url) = str_option(options, s3_constants::AWS_ENDPOINT_URL) { + sdk_config.to_builder().endpoint_url(endpoint_url).build() + } else { + sdk_config + }; + Ok(Self { + virtual_hosted_style_request, + locking_provider: str_option(options, s3_constants::AWS_S3_LOCKING_PROVIDER), + s3_pool_idle_timeout: Duration::from_secs(s3_pool_idle_timeout), + sts_pool_idle_timeout: Duration::from_secs(sts_pool_idle_timeout), + s3_get_internal_server_error_retries, + allow_unsafe_rename, + extra_opts, + sdk_config, + }) + } + + pub fn endpoint_url(&self) -> Option<&str> { + self.sdk_config.endpoint_url() + } + + pub fn region(&self) -> Option<&Region> { + self.sdk_config.region() + } + + fn u64_or_default(map: &HashMap, key: &str, default: u64) -> u64 { + str_option(map, key) + .and_then(|v| v.parse().ok()) + .unwrap_or(default) + } + + fn ensure_env_var(map: &HashMap, key: &str) { + if let Some(val) = str_option(map, key) { + std::env::set_var(key, val); + } + } + + pub fn try_default() -> DeltaResult { + Self::from_map(&HashMap::new()) + } +} + +fn execute_sdk_future + Send + 'static>( + future: F, +) -> DeltaResult { + match tokio::runtime::Handle::try_current() { + Ok(handle) => match handle.runtime_flavor() { + tokio::runtime::RuntimeFlavor::MultiThread => { + Ok(tokio::task::block_in_place(move || handle.block_on(future))) + } + _ => { + let mut cfg: Option = None; + std::thread::scope(|scope| { + scope.spawn(|| { + cfg = Some(handle.block_on(future)); + }); + }); + cfg.ok_or(deltalake_core::DeltaTableError::ObjectStore { + source: ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(DynamoDbConfigError::InitializationError), + }, + }) + } + }, + Err(_) => { + let runtime = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .expect("a tokio runtime is required by the AWS sdk"); + Ok(runtime.block_on(future)) + } + } +} + +/// An S3 implementation of the [ObjectStore] trait +pub struct S3StorageBackend { + inner: ObjectStoreRef, + /// Whether allowed to performance rename_if_not_exist as rename + allow_unsafe_rename: bool, +} + +impl std::fmt::Display for S3StorageBackend { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "S3StorageBackend") + } +} + +impl S3StorageBackend { + /// Creates a new S3StorageBackend. + /// + /// Options are described in [s3_constants]. + pub fn try_new(storage: ObjectStoreRef, allow_unsafe_rename: bool) -> ObjectStoreResult { + Ok(Self { + inner: storage, + allow_unsafe_rename, + }) + } +} + +impl std::fmt::Debug for S3StorageBackend { + fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(fmt, "S3StorageBackend") + } +} + +#[async_trait::async_trait] +impl ObjectStore for S3StorageBackend { + async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult { + self.inner.put(location, bytes).await + } + + async fn put_opts( + &self, + location: &Path, + bytes: Bytes, + options: PutOptions, + ) -> ObjectStoreResult { + self.inner.put_opts(location, bytes, options).await + } + + async fn get(&self, location: &Path) -> ObjectStoreResult { + self.inner.get(location).await + } + + async fn get_opts(&self, location: &Path, options: GetOptions) -> ObjectStoreResult { + self.inner.get_opts(location, options).await + } + + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + self.inner.get_range(location, range).await + } + + async fn head(&self, location: &Path) -> ObjectStoreResult { + self.inner.head(location).await + } + + async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { + self.inner.delete(location).await + } + + fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list(prefix) + } + + fn list_with_offset( + &self, + prefix: Option<&Path>, + offset: &Path, + ) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list_with_offset(prefix, offset) + } + + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { + self.inner.list_with_delimiter(prefix).await + } + + async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + self.inner.copy(from, to).await + } + + async fn copy_if_not_exists(&self, _from: &Path, _to: &Path) -> ObjectStoreResult<()> { + todo!() + } + + async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + if self.allow_unsafe_rename { + self.inner.rename(from, to).await + } else { + Err(ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(crate::errors::LockClientError::LockClientRequired), + }) + } + } + + async fn put_multipart( + &self, + location: &Path, + ) -> ObjectStoreResult<(MultipartId, Box)> { + self.inner.put_multipart(location).await + } + + async fn abort_multipart( + &self, + location: &Path, + multipart_id: &MultipartId, + ) -> ObjectStoreResult<()> { + self.inner.abort_multipart(location, multipart_id).await + } +} + +/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. +/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. +/// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. +pub mod s3_constants { + /// Custom S3 endpoint. + pub const AWS_ENDPOINT_URL: &str = "AWS_ENDPOINT_URL"; + /// The AWS region. + pub const AWS_REGION: &str = "AWS_REGION"; + /// The AWS profile. + pub const AWS_PROFILE: &str = "AWS_PROFILE"; + /// The AWS_ACCESS_KEY_ID to use for S3. + pub const AWS_ACCESS_KEY_ID: &str = "AWS_ACCESS_KEY_ID"; + /// The AWS_SECRET_ACCESS_KEY to use for S3. + pub const AWS_SECRET_ACCESS_KEY: &str = "AWS_SECRET_ACCESS_KEY"; + /// The AWS_SESSION_TOKEN to use for S3. + pub const AWS_SESSION_TOKEN: &str = "AWS_SESSION_TOKEN"; + /// Uses either "path" (the default) or "virtual", which turns on + /// [virtual host addressing](http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). + pub const AWS_S3_ADDRESSING_STYLE: &str = "AWS_S3_ADDRESSING_STYLE"; + /// Locking provider to use for safe atomic rename. + /// `dynamodb` is currently the only supported locking provider. + /// If not set, safe atomic rename is not available. + pub const AWS_S3_LOCKING_PROVIDER: &str = "AWS_S3_LOCKING_PROVIDER"; + /// The role to assume for S3 writes. + pub const AWS_S3_ASSUME_ROLE_ARN: &str = "AWS_S3_ASSUME_ROLE_ARN"; + /// The role session name to use when a role is assumed. If not provided a random session name is generated. + pub const AWS_S3_ROLE_SESSION_NAME: &str = "AWS_S3_ROLE_SESSION_NAME"; + /// The `pool_idle_timeout` option of aws http client. Has to be lower than 20 seconds, which is + /// default S3 server timeout . + /// However, since rusoto uses hyper as a client, its default timeout is 90 seconds + /// . + /// Hence, the `connection closed before message completed` could occur. + /// To avoid that, the default value of this setting is 15 seconds if it's not set otherwise. + pub const AWS_S3_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_S3_POOL_IDLE_TIMEOUT_SECONDS"; + /// The `pool_idle_timeout` for the as3_constants sts client. See + /// the reasoning in `AWS_S3_POOL_IDLE_TIMEOUT_SECONDS`. + pub const AWS_STS_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_STS_POOL_IDLE_TIMEOUT_SECONDS"; + /// The number of retries for S3 GET requests failed with 500 Internal Server Error. + pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = + "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES"; + /// The web identity token file to use when using a web identity provider. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; + /// The role name to use for web identity. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; + /// The role session name to use for web identity. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; + /// Allow http connections - mainly useful for integration tests + pub const AWS_ALLOW_HTTP: &str = "AWS_ALLOW_HTTP"; + + /// If set to "true", allows creating commits without concurrent writer protection. + /// Only safe if there is one writer to a given table. + pub const AWS_S3_ALLOW_UNSAFE_RENAME: &str = "AWS_S3_ALLOW_UNSAFE_RENAME"; + + /// The list of option keys owned by the S3 module. + /// Option keys not contained in this list will be added to the `extra_opts` + /// field of [crate::storage::s3::S3StorageOptions]. + pub const S3_OPTS: &[&str] = &[ + AWS_ENDPOINT_URL, + AWS_REGION, + AWS_PROFILE, + AWS_ACCESS_KEY_ID, + AWS_SECRET_ACCESS_KEY, + AWS_SESSION_TOKEN, + AWS_S3_LOCKING_PROVIDER, + AWS_S3_ASSUME_ROLE_ARN, + AWS_S3_ROLE_SESSION_NAME, + AWS_WEB_IDENTITY_TOKEN_FILE, + AWS_ROLE_ARN, + AWS_ROLE_SESSION_NAME, + AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, + AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, + AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + ]; +} + +pub(crate) fn str_option(map: &HashMap, key: &str) -> Option { + map.get(key) + .map_or_else(|| std::env::var(key).ok(), |v| Some(v.to_owned())) +} + +#[cfg(test)] +mod tests { + use super::*; + + use maplit::hashmap; + use serial_test::serial; + + struct ScopedEnv { + vars: HashMap, + } + + impl ScopedEnv { + pub fn new() -> Self { + let vars = std::env::vars_os().collect(); + Self { vars } + } + + pub fn run(mut f: impl FnMut() -> T) -> T { + let _env_scope = Self::new(); + f() + } + } + + impl Drop for ScopedEnv { + fn drop(&mut self) { + let to_remove: Vec<_> = std::env::vars_os() + .map(|kv| kv.0) + .filter(|k| !self.vars.contains_key(k)) + .collect(); + for k in to_remove { + std::env::remove_var(k); + } + for (key, value) in self.vars.drain() { + std::env::set_var(key, value); + } + } + } + + fn clear_env_of_aws_keys() { + let keys_to_clear = std::env::vars().filter_map(|(k, _v)| { + if AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()).is_ok() { + Some(k) + } else { + None + } + }); + + for k in keys_to_clear { + std::env::remove_var(k); + } + } + + #[test] + #[serial] + fn storage_options_default_test() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + + std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "http://localhost"); + std::env::set_var(s3_constants::AWS_REGION, "us-west-1"); + std::env::set_var(s3_constants::AWS_PROFILE, "default"); + std::env::set_var(s3_constants::AWS_ACCESS_KEY_ID, "default_key_id"); + std::env::set_var(s3_constants::AWS_SECRET_ACCESS_KEY, "default_secret_key"); + std::env::set_var(s3_constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + std::env::set_var( + s3_constants::AWS_S3_ASSUME_ROLE_ARN, + "arn:aws:iam::123456789012:role/some_role", + ); + std::env::set_var(s3_constants::AWS_S3_ROLE_SESSION_NAME, "session_name"); + std::env::set_var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE, "token_file"); + + let options = S3StorageOptions::try_default().unwrap(); + assert_eq!( + S3StorageOptions { + sdk_config: SdkConfig::builder() + .endpoint_url("http://localhost".to_string()) + .region(Region::from_static("us-west-1")) + .build(), + virtual_hosted_style_request: false, + locking_provider: Some("dynamodb".to_string()), + s3_pool_idle_timeout: Duration::from_secs(15), + sts_pool_idle_timeout: Duration::from_secs(10), + s3_get_internal_server_error_retries: 10, + extra_opts: HashMap::new(), + allow_unsafe_rename: false, + }, + options + ); + }); + } + + #[test] + #[serial] + fn storage_options_with_only_region_and_credentials() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + std::env::remove_var(s3_constants::AWS_ENDPOINT_URL); + let options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_REGION.to_string() => "eu-west-1".to_string(), + s3_constants::AWS_ACCESS_KEY_ID.to_string() => "test".to_string(), + s3_constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), + }) + .unwrap(); + + let mut expected = S3StorageOptions::try_default().unwrap(); + expected.sdk_config = SdkConfig::builder() + .region(Region::from_static("eu-west-1")) + .build(); + assert_eq!(expected, options); + }); + } + + #[test] + #[serial] + fn storage_options_from_map_test() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + let options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_ENDPOINT_URL.to_string() => "http://localhost:1234".to_string(), + s3_constants::AWS_REGION.to_string() => "us-west-2".to_string(), + s3_constants::AWS_PROFILE.to_string() => "default".to_string(), + s3_constants::AWS_S3_ADDRESSING_STYLE.to_string() => "virtual".to_string(), + s3_constants::AWS_S3_LOCKING_PROVIDER.to_string() => "another_locking_provider".to_string(), + s3_constants::AWS_S3_ASSUME_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/another_role".to_string(), + s3_constants::AWS_S3_ROLE_SESSION_NAME.to_string() => "another_session_name".to_string(), + s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "another_token_file".to_string(), + s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "1".to_string(), + s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "2".to_string(), + s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES.to_string() => "3".to_string(), + s3_constants::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), + s3_constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), + }).unwrap(); + + assert_eq!( + S3StorageOptions { + sdk_config: SdkConfig::builder() + .endpoint_url("http://localhost:1234".to_string()) + .region(Region::from_static("us-west-2")) + .build(), + virtual_hosted_style_request: true, + locking_provider: Some("another_locking_provider".to_string()), + s3_pool_idle_timeout: Duration::from_secs(1), + sts_pool_idle_timeout: Duration::from_secs(2), + s3_get_internal_server_error_retries: 3, + extra_opts: hashmap! { + s3_constants::AWS_S3_ADDRESSING_STYLE.to_string() => "virtual".to_string() + }, + allow_unsafe_rename: false, + }, + options + ); + }); + } + + #[test] + #[serial] + fn storage_options_mixed_test() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "http://localhost"); + std::env::set_var(s3_constants::AWS_REGION, "us-west-1"); + std::env::set_var(s3_constants::AWS_PROFILE, "default"); + std::env::set_var(s3_constants::AWS_ACCESS_KEY_ID, "wrong_key_id"); + std::env::set_var(s3_constants::AWS_SECRET_ACCESS_KEY, "wrong_secret_key"); + std::env::set_var(s3_constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + std::env::set_var( + s3_constants::AWS_S3_ASSUME_ROLE_ARN, + "arn:aws:iam::123456789012:role/some_role", + ); + std::env::set_var(s3_constants::AWS_S3_ROLE_SESSION_NAME, "session_name"); + std::env::set_var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE, "token_file"); + + std::env::set_var(s3_constants::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, "1"); + std::env::set_var(s3_constants::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, "2"); + std::env::set_var(s3_constants::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, "3"); + let options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_ACCESS_KEY_ID.to_string() => "test_id_mixed".to_string(), + s3_constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret_mixed".to_string(), + s3_constants::AWS_REGION.to_string() => "us-west-2".to_string(), + "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES".to_string() => "3".to_string(), + }) + .unwrap(); + + assert_eq!( + S3StorageOptions { + sdk_config: SdkConfig::builder() + .endpoint_url("http://localhost".to_string()) + .region(Region::from_static("us-west-2")) + .build(), + virtual_hosted_style_request: false, + locking_provider: Some("dynamodb".to_string()), + s3_pool_idle_timeout: Duration::from_secs(1), + sts_pool_idle_timeout: Duration::from_secs(2), + s3_get_internal_server_error_retries: 3, + extra_opts: hashmap! {}, + allow_unsafe_rename: false, + }, + options + ); + }); + } + + #[test] + #[serial] + fn storage_options_web_identity_test() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + let _options = S3StorageOptions::from_map(&hashmap! { + s3_constants::AWS_REGION.to_string() => "eu-west-1".to_string(), + s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "web_identity_token_file".to_string(), + s3_constants::AWS_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/web_identity_role".to_string(), + s3_constants::AWS_ROLE_SESSION_NAME.to_string() => "web_identity_session_name".to_string(), + }).unwrap(); + + assert_eq!( + "eu-west-1", + std::env::var(s3_constants::AWS_REGION).unwrap() + ); + + assert_eq!( + "web_identity_token_file", + std::env::var(s3_constants::AWS_WEB_IDENTITY_TOKEN_FILE).unwrap() + ); + + assert_eq!( + "arn:aws:iam::123456789012:role/web_identity_role", + std::env::var(s3_constants::AWS_ROLE_ARN).unwrap() + ); + + assert_eq!( + "web_identity_session_name", + std::env::var(s3_constants::AWS_ROLE_SESSION_NAME).unwrap() + ); + }); + } + + #[test] + #[serial] + fn when_merging_with_env_unsupplied_options_are_added() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + let raw_options = hashmap! {}; + + std::env::set_var(s3_constants::AWS_ACCESS_KEY_ID, "env_key"); + std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "env_key"); + std::env::set_var(s3_constants::AWS_SECRET_ACCESS_KEY, "env_key"); + std::env::set_var(s3_constants::AWS_REGION, "env_key"); + + let combined_options = + S3ObjectStoreFactory {}.with_env_s3(&StorageOptions(raw_options)); + + assert_eq!(combined_options.0.len(), 4); + + for v in combined_options.0.values() { + assert_eq!(v, "env_key"); + } + }); + } + + #[tokio::test] + #[serial] + async fn when_merging_with_env_supplied_options_take_precedence() { + ScopedEnv::run(|| { + clear_env_of_aws_keys(); + let raw_options = hashmap! { + "AWS_ACCESS_KEY_ID".to_string() => "options_key".to_string(), + "AWS_ENDPOINT_URL".to_string() => "options_key".to_string(), + "AWS_SECRET_ACCESS_KEY".to_string() => "options_key".to_string(), + "AWS_REGION".to_string() => "options_key".to_string() + }; + + std::env::set_var("aws_access_key_id", "env_key"); + std::env::set_var("aws_endpoint", "env_key"); + std::env::set_var("aws_secret_access_key", "env_key"); + std::env::set_var("aws_region", "env_key"); + + let combined_options = + S3ObjectStoreFactory {}.with_env_s3(&StorageOptions(raw_options)); + + for v in combined_options.0.values() { + assert_eq!(v, "options_key"); + } + }); + } +} diff --git a/crates/aws/tests/common.rs b/crates/aws/tests/common.rs new file mode 100644 index 0000000000..01aa505b1b --- /dev/null +++ b/crates/aws/tests/common.rs @@ -0,0 +1,167 @@ +use chrono::Utc; +use deltalake_aws::constants; +use deltalake_aws::register_handlers; +use deltalake_aws::storage::*; +use deltalake_test::utils::*; +use rand::Rng; +use std::process::{Command, ExitStatus, Stdio}; + +#[derive(Clone, Debug)] +pub struct S3Integration { + bucket_name: String, +} + +impl Default for S3Integration { + fn default() -> Self { + register_handlers(None); + Self { + bucket_name: format!("test-delta-table-{}", Utc::now().timestamp()), + } + } +} + +impl StorageIntegration for S3Integration { + /// Create a new bucket + fn create_bucket(&self) -> std::io::Result { + Self::create_lock_table()?; + let mut child = Command::new("aws") + .args(["s3", "mb", &self.root_uri()]) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + fn bucket_name(&self) -> String { + self.bucket_name.clone() + } + + fn root_uri(&self) -> String { + format!("s3://{}", &self.bucket_name()) + } + + /// prepare_env + fn prepare_env(&self) { + set_env_if_not_set( + constants::LOCK_TABLE_KEY_NAME, + format!("delta_log_it_{}", rand::thread_rng().gen::()), + ); + match std::env::var(s3_constants::AWS_ENDPOINT_URL).ok() { + Some(endpoint_url) if endpoint_url.to_lowercase() == "none" => { + std::env::remove_var(s3_constants::AWS_ENDPOINT_URL) + } + Some(_) => (), + None => std::env::set_var(s3_constants::AWS_ENDPOINT_URL, "http://localhost:4566"), + } + set_env_if_not_set(s3_constants::AWS_ACCESS_KEY_ID, "deltalake"); + set_env_if_not_set(s3_constants::AWS_SECRET_ACCESS_KEY, "weloverust"); + set_env_if_not_set(s3_constants::AWS_REGION, "us-east-1"); + set_env_if_not_set(s3_constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + } + + /// copy directory + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + let destination = format!("{}/{destination}", self.root_uri()); + let mut child = Command::new("aws") + .args(["s3", "cp", source, &destination, "--recursive"]) + .spawn() + .expect("aws command is installed"); + child.wait() + } +} + +impl S3Integration { + /// delete bucket + fn delete_bucket(bucket_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("aws") + .args(["s3", "rb", bucket_name.as_ref(), "--force"]) + .spawn() + .expect("aws command is installed"); + child.wait() + } + fn create_dynamodb_table( + table_name: &str, + attr_definitions: &[&str], + key_schema: &[&str], + ) -> std::io::Result { + let args = [ + "dynamodb", + "create-table", + "--table-name", + &table_name, + "--provisioned-throughput", + "ReadCapacityUnits=1,WriteCapacityUnits=1", + "--attribute-definitions", + ]; + let mut child = Command::new("aws") + .args(args) + .args(attr_definitions.iter()) + .arg("--key-schema") + .args(key_schema) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + let status = child.wait()?; + Self::wait_for_table(table_name)?; + Ok(status) + } + + fn find_subsequence(haystack: &[u8], needle: &[u8]) -> Option { + haystack + .windows(needle.len()) + .position(|window| window == needle) + } + + fn wait_for_table(table_name: &str) -> std::io::Result<()> { + let args = ["dynamodb", "describe-table", "--table-name", &table_name]; + loop { + let output = Command::new("aws") + .args(args) + .output() + .expect("aws command is installed"); + if Self::find_subsequence(&output.stdout, "CREATING".as_bytes()).is_some() { + std::thread::sleep(std::time::Duration::from_millis(200)); + continue; + } else { + return Ok(()); + } + } + } + + pub fn create_lock_table() -> std::io::Result { + let table_name = + std::env::var(constants::LOCK_TABLE_KEY_NAME).unwrap_or_else(|_| "delta_log".into()); + Self::create_dynamodb_table( + &table_name, + &[ + "AttributeName=tablePath,AttributeType=S", + "AttributeName=fileName,AttributeType=S", + ], + &[ + "AttributeName=tablePath,KeyType=HASH", + "AttributeName=fileName,KeyType=RANGE", + ], + ) + } + + fn delete_dynamodb_table(table_name: &str) -> std::io::Result { + let mut child = Command::new("aws") + .args(["dynamodb", "delete-table", "--table-name", &table_name]) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + pub fn delete_lock_table() -> std::io::Result { + let table_name = + std::env::var(constants::LOCK_TABLE_KEY_NAME).unwrap_or_else(|_| "delta_log".into()); + Self::delete_dynamodb_table(&table_name) + } +} + +impl Drop for S3Integration { + fn drop(&mut self) { + Self::delete_bucket(self.root_uri()).expect("Failed to drop bucket"); + Self::delete_lock_table().expect("Failed to delete lock table"); + } +} diff --git a/crates/aws/tests/integration_read.rs b/crates/aws/tests/integration_read.rs new file mode 100644 index 0000000000..0a610b0bf9 --- /dev/null +++ b/crates/aws/tests/integration_read.rs @@ -0,0 +1,26 @@ +#![cfg(feature = "integration_test")] + +use deltalake_test::read::{read_table_paths, test_read_tables}; +use deltalake_test::utils::*; +use serial_test::serial; + +mod common; +use common::*; + +static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; +/// TEST_PREFIXES as they should appear in object stores. +static TEST_PREFIXES_ENCODED: &[&str] = &["my%20table", "%E4%BD%A0%E5%A5%BD/%F0%9F%98%8A"]; + +#[tokio::test] +#[serial] +async fn test_read_tables_aws() -> TestResult { + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; + + test_read_tables(&context).await?; + + for (prefix, prefix_encoded) in TEST_PREFIXES.iter().zip(TEST_PREFIXES_ENCODED.iter()) { + read_table_paths(&context, prefix, prefix_encoded).await?; + } + + Ok(()) +} diff --git a/crates/deltalake-core/tests/integration_s3_dynamodb.rs b/crates/aws/tests/integration_s3_dynamodb.rs similarity index 72% rename from crates/deltalake-core/tests/integration_s3_dynamodb.rs rename to crates/aws/tests/integration_s3_dynamodb.rs index f347e45efd..cf69818966 100644 --- a/crates/deltalake-core/tests/integration_s3_dynamodb.rs +++ b/crates/aws/tests/integration_s3_dynamodb.rs @@ -1,32 +1,32 @@ //! Integration test to verify correct behavior of S3 DynamoDb locking. //! It inspects the state of the locking table after each operation. -#![cfg(all( - feature = "integration_test", - any(feature = "s3", feature = "s3-native-tls") -))] +#![cfg(feature = "integration_test")] use std::collections::HashMap; use std::time::{Duration, SystemTime, UNIX_EPOCH}; -use deltalake_aws::{CommitEntry, DynamoDbLockClient}; +use aws_config::SdkConfig; +use aws_sdk_dynamodb::types::BillingMode; +use deltalake_aws::logstore::{RepairLogEntryResult, S3DynamoDbLogStore}; +use deltalake_aws::storage::S3StorageOptions; +use deltalake_aws::{CommitEntry, DynamoDbConfig, DynamoDbLockClient}; use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; -use deltalake_core::logstore::s3::{RepairLogEntryResult, S3DynamoDbLogStore}; use deltalake_core::logstore::LogStore; -use deltalake_core::operations::transaction::{commit, prepare_commit}; +use deltalake_core::operations::transaction::{CommitBuilder, PreparedCommit}; +use deltalake_core::parquet::file::metadata; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use deltalake_core::storage::commit_uri_from_version; -use deltalake_core::storage::config::StorageOptions; -use deltalake_core::storage::s3::S3StorageOptions; +use deltalake_core::storage::StorageOptions; use deltalake_core::table::builder::ensure_table_uri; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestTables}; use deltalake_core::{DeltaOps, DeltaTable, DeltaTableBuilder}; +use deltalake_test::utils::*; use lazy_static::lazy_static; use object_store::path::Path; use serde_json::Value; use serial_test::serial; -#[allow(dead_code)] -mod fs_common; +mod common; +use common::*; pub type TestResult = Result>; @@ -34,32 +34,56 @@ lazy_static! { static ref OPTIONS: HashMap = maplit::hashmap! { "allow_http".to_owned() => "true".to_owned(), }; - static ref S3_OPTIONS: S3StorageOptions = S3StorageOptions::from_map(&OPTIONS); + static ref S3_OPTIONS: S3StorageOptions = S3StorageOptions::from_map(&OPTIONS).unwrap(); } fn make_client() -> TestResult { + let options: S3StorageOptions = S3StorageOptions::try_default().unwrap(); Ok(DynamoDbLockClient::try_new( + &options.sdk_config, + None, None, None, - S3_OPTIONS.region.clone(), - false, )?) } #[test] #[serial] -fn client_config_picks_up_lock_table_name() -> TestResult<()> { - let _context = IntegrationContext::new(StorageIntegration::Amazon)?; - assert!(make_client()? - .get_lock_table_name() - .starts_with("delta_log_it_")); +fn client_configs_via_env_variables() -> TestResult<()> { + std::env::set_var( + deltalake_aws::constants::MAX_ELAPSED_REQUEST_TIME_KEY_NAME, + "64", + ); + std::env::set_var( + deltalake_aws::constants::LOCK_TABLE_KEY_NAME, + "some_table".to_owned(), + ); + std::env::set_var( + deltalake_aws::constants::BILLING_MODE_KEY_NAME, + "PAY_PER_REQUEST".to_owned(), + ); + let client = make_client()?; + let config = client.get_dynamodb_config(); + let options: S3StorageOptions = S3StorageOptions::try_default().unwrap(); + assert_eq!( + DynamoDbConfig { + billing_mode: BillingMode::PayPerRequest, + lock_table_name: "some_table".to_owned(), + max_elapsed_request_time: Duration::from_secs(64), + sdk_config: options.sdk_config, + }, + *config, + ); + std::env::remove_var(deltalake_aws::constants::LOCK_TABLE_KEY_NAME); + std::env::remove_var(deltalake_aws::constants::MAX_ELAPSED_REQUEST_TIME_KEY_NAME); + std::env::remove_var(deltalake_aws::constants::BILLING_MODE_KEY_NAME); Ok(()) } #[tokio::test] #[serial] async fn get_missing_item() -> TestResult<()> { - let _context = IntegrationContext::new(StorageIntegration::Amazon)?; + let _context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let version = i64::MAX; let result = client @@ -75,7 +99,7 @@ async fn get_missing_item() -> TestResult<()> { #[tokio::test] #[serial] async fn test_append() -> TestResult<()> { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let table = prepare_table(&context, "delta01").await?; validate_lock_table_state(&table, 0).await?; append_to_table("datav01.parquet", &table, None).await?; @@ -86,7 +110,7 @@ async fn test_append() -> TestResult<()> { #[tokio::test] #[serial] async fn test_repair_commit_entry() -> TestResult<()> { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let table = prepare_table(&context, "repair_needed").await?; let options: StorageOptions = OPTIONS.clone().into(); @@ -135,7 +159,7 @@ async fn test_repair_commit_entry() -> TestResult<()> { #[tokio::test] #[serial] async fn test_repair_on_update() -> TestResult<()> { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let mut table = prepare_table(&context, "repair_on_update").await?; let _entry = create_incomplete_commit_entry(&table, 1, "unfinished_commit").await?; table.update().await?; @@ -145,16 +169,32 @@ async fn test_repair_on_update() -> TestResult<()> { Ok(()) } +#[tokio::test] +#[serial] +async fn test_repair_on_load() -> TestResult<()> { + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; + let mut table = prepare_table(&context, "repair_on_update").await?; + let _entry = create_incomplete_commit_entry(&table, 1, "unfinished_commit").await?; + table.load_version(1).await?; + // table should fix the broken entry while loading a specific version + assert_eq!(table.version(), 1); + validate_lock_table_state(&table, 1).await?; + Ok(()) +} + const WORKERS: i64 = 3; -const COMMITS: i64 = 5; +const COMMITS: i64 = 15; -#[tokio::test] +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] #[serial] async fn test_concurrent_writers() -> TestResult<()> { // Goal: a test with multiple writers, very similar to `integration_concurrent_writes` - let context = IntegrationContext::new(StorageIntegration::Amazon)?; + let context = IntegrationContext::new(Box::new(S3Integration::default()))?; + println!(">>> preparing table"); let table = prepare_table(&context, "concurrent_writes").await?; + println!(">>> table prepared"); let table_uri = table.table_uri(); + println!("Starting workers on {table_uri}"); let mut workers = Vec::new(); for w in 0..WORKERS { @@ -170,7 +210,9 @@ async fn test_concurrent_writers() -> TestResult<()> { for f in futures { map.extend(f.await?); } + validate_lock_table_state(&table, WORKERS * COMMITS).await?; + Ok(()) } @@ -187,6 +229,7 @@ impl Worker { .load() .await .unwrap(); + println!("Loaded table in worker: {table:?}"); Self { table, name } } @@ -219,18 +262,18 @@ async fn create_incomplete_commit_entry( tag: &str, ) -> TestResult { let actions = vec![add_action(tag)]; - let temp_path = prepare_commit( - table.object_store().as_ref(), - &DeltaOperation::Write { - mode: SaveMode::Append, - partition_by: None, - predicate: None, - }, - &actions, - None, - ) - .await?; - let commit_entry = CommitEntry::new(version, temp_path); + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + let prepared = CommitBuilder::default() + .with_actions(actions) + .build(Some(table.snapshot()?), table.log_store(), operation)? + .into_prepared_commit_future() + .await?; + + let commit_entry = CommitEntry::new(version, prepared.path().to_owned()); make_client()? .put_commit_entry(&table.table_uri(), &commit_entry) .await?; @@ -241,12 +284,11 @@ fn add_action(name: &str) -> Action { let ts = (SystemTime::now() - Duration::from_secs(1800)) .duration_since(UNIX_EPOCH) .unwrap() - .as_secs(); - Action::Add(Add { + .as_millis(); + Add { path: format!("{}.parquet", name), size: 396, partition_values: HashMap::new(), - partition_values_parsed: None, modification_time: ts as i64, data_change: true, stats: None, @@ -256,11 +298,11 @@ fn add_action(name: &str) -> Action { base_row_id: None, default_row_commit_version: None, clustering_provider: None, - }) + } + .into() } async fn prepare_table(context: &IntegrationContext, table_name: &str) -> TestResult { - make_client()?.try_create_lock_table().await?; let table_name = format!("{}_{}", table_name, uuid::Uuid::new_v4()); let table_uri = context.uri_for_table(TestTables::Custom(table_name.to_owned())); let schema = StructType::new(vec![StructField::new( @@ -272,11 +314,13 @@ async fn prepare_table(context: &IntegrationContext, table_name: &str) -> TestRe .with_allow_http(true) .with_storage_options(OPTIONS.clone()) .build()?; + println!("table built: {table:?}"); // create delta table let table = DeltaOps(table) .create() .with_columns(schema.fields().clone()) .await?; + println!("table created: {table:?}"); Ok(table) } @@ -291,15 +335,12 @@ async fn append_to_table( predicate: None, }; let actions = vec![add_action(name)]; - let version = commit( - table.log_store().as_ref(), - &actions, - operation, - &table.state, - metadata, - ) - .await - .unwrap(); + let version = CommitBuilder::default() + .with_actions(actions) + .with_app_metadata(metadata.unwrap_or_default()) + .build(Some(table.snapshot()?), table.log_store(), operation)? + .await? + .version(); Ok(version) } diff --git a/crates/deltalake-core/tests/repair_s3_rename_test.rs b/crates/aws/tests/repair_s3_rename_test.rs similarity index 89% rename from crates/deltalake-core/tests/repair_s3_rename_test.rs rename to crates/aws/tests/repair_s3_rename_test.rs index ecab792f39..68d8727ebe 100644 --- a/crates/deltalake-core/tests/repair_s3_rename_test.rs +++ b/crates/aws/tests/repair_s3_rename_test.rs @@ -1,16 +1,14 @@ -#![cfg(all( - any(feature = "s3", feature = "s3-native-tls"), - feature = "integration_test" -))] +#![cfg(feature = "integration_test")] + use bytes::Bytes; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration}; -use deltalake_core::{storage::s3::S3StorageBackend, DeltaTableBuilder, ObjectStore}; -use futures::stream::BoxStream; -use object_store::path::Path; -use object_store::{ +use deltalake_aws::storage::S3StorageBackend; +use deltalake_core::storage::object_store::{ DynObjectStore, Error as ObjectStoreError, GetOptions, GetResult, ListResult, MultipartId, - ObjectMeta, Result as ObjectStoreResult, + ObjectMeta, PutOptions, PutResult, Result as ObjectStoreResult, }; +use deltalake_core::{DeltaTableBuilder, ObjectStore, Path}; +use deltalake_test::utils::IntegrationContext; +use futures::stream::BoxStream; use serial_test::serial; use std::ops::Range; use std::sync::{Arc, Mutex}; @@ -18,6 +16,9 @@ use tokio::io::AsyncWrite; use tokio::task::JoinHandle; use tokio::time::Duration; +mod common; +use common::*; + #[tokio::test(flavor = "multi_thread")] #[serial] #[ignore = "currently tests are hanging"] @@ -43,7 +44,7 @@ async fn repair_when_worker_pauses_after_rename_test() { async fn run_repair_test_case(path: &str, pause_copy: bool) -> Result<(), ObjectStoreError> { std::env::set_var("AWS_S3_LOCKING_PROVIDER", "dynamodb"); std::env::set_var("DYNAMO_LOCK_LEASE_DURATION", "2"); - let context = IntegrationContext::new(StorageIntegration::Amazon).unwrap(); + let context = IntegrationContext::new(Box::new(S3Integration::default())).unwrap(); let root_path = Path::from(path); let src1 = root_path.child("src1"); @@ -165,10 +166,19 @@ impl ObjectStore for DelayedObjectStore { self.delete(from).await } - async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult<()> { + async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult { self.inner.put(location, bytes).await } + async fn put_opts( + &self, + location: &Path, + bytes: Bytes, + options: PutOptions, + ) -> ObjectStoreResult { + self.inner.put_opts(location, bytes, options).await + } + async fn get(&self, location: &Path) -> ObjectStoreResult { self.inner.get(location).await } @@ -189,11 +199,16 @@ impl ObjectStore for DelayedObjectStore { self.inner.delete(location).await } - async fn list( + fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list(prefix) + } + + fn list_with_offset( &self, prefix: Option<&Path>, - ) -> ObjectStoreResult>> { - self.inner.list(prefix).await + offset: &Path, + ) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list_with_offset(prefix, offset) } async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { diff --git a/crates/azure/Cargo.toml b/crates/azure/Cargo.toml new file mode 100644 index 0000000000..ca21ed0f36 --- /dev/null +++ b/crates/azure/Cargo.toml @@ -0,0 +1,38 @@ +[package] +name = "deltalake-azure" +version = "0.1.0" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +deltalake-core = { version = "0.17.0", path = "../core" } +lazy_static = "1" + +# workspace depenndecies +async-trait = { workspace = true } +bytes = { workspace = true } +futures = { workspace = true } +tracing = { workspace = true } +object_store = { workspace = true, features = ["azure"]} +thiserror = { workspace = true } +tokio = { workspace = true } +regex = { workspace = true } +url = { workspace = true } + +[dev-dependencies] +chrono = { workspace = true } +serial_test = "3" +deltalake-test = { path = "../test" } +pretty_env_logger = "0.5.0" +rand = "0.8" +serde_json = { workspace = true } + +[features] +integration_test = [] diff --git a/crates/deltalake-core/src/storage/config/azure.rs b/crates/azure/src/config.rs similarity index 83% rename from crates/deltalake-core/src/storage/config/azure.rs rename to crates/azure/src/config.rs index e556b9eb6e..d30272768e 100644 --- a/crates/deltalake-core/src/storage/config/azure.rs +++ b/crates/azure/src/config.rs @@ -8,9 +8,9 @@ use std::collections::{hash_map::Entry, HashMap}; use std::str::FromStr; use object_store::azure::AzureConfigKey; -use object_store::Error; +use object_store::Error as ObjectStoreError; -use crate::{DeltaResult, DeltaTableError}; +use crate::error::Result; lazy_static::lazy_static! { static ref CREDENTIAL_KEYS: Vec = @@ -35,6 +35,7 @@ enum AzureCredential { /// Authorizing with secret ClientSecret, /// Using a shared access signature + #[allow(dead_code)] ManagedIdentity, /// Using a shared access signature SasKey, @@ -42,40 +43,8 @@ enum AzureCredential { WorkloadIdentity, } -impl FromStr for AzureCredential { - type Err = DeltaTableError; - - fn from_str(s: &str) -> Result { - match s { - "access_key" => Ok(AzureCredential::AccessKey), - "bearer_token" => Ok(AzureCredential::BearerToken), - "client_secret" => Ok(AzureCredential::ClientSecret), - "managed_identity" => Ok(AzureCredential::ManagedIdentity), - "workload_identity" => Ok(AzureCredential::WorkloadIdentity), - "sas_key" => Ok(AzureCredential::SasKey), - _ => Err(DeltaTableError::Generic(format!( - "Cannot parse AzureCredential variant from {}", - s - ))), - } - } -} - -impl AsRef for AzureCredential { - fn as_ref(&self) -> &str { - match self { - Self::AccessKey => "access_key", - Self::BearerToken => "bearer_token", - Self::ClientSecret => "client_secret", - Self::ManagedIdentity => "managed_identity", - Self::SasKey => "sas_key", - Self::WorkloadIdentity => "workload_identity", - } - } -} - impl AzureCredential { - /// Reys required for config + /// required configuration keys for variant fn keys(&self) -> Vec { match self { Self::AccessKey => Vec::from_iter([AzureConfigKey::AccessKey]), @@ -110,7 +79,7 @@ impl AzureConfigHelper { /// Create a new [`ConfigHelper`] pub fn try_new( config: impl IntoIterator, impl Into)>, - ) -> DeltaResult { + ) -> Result { let mut env_config = HashMap::new(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { @@ -126,7 +95,7 @@ impl AzureConfigHelper { config: config .into_iter() .map(|(key, value)| Ok((AzureConfigKey::from_str(key.as_ref())?, value.into()))) - .collect::>()?, + .collect::>()?, env_config, priority: Vec::from_iter([ AzureCredential::AccessKey, @@ -156,7 +125,7 @@ impl AzureConfigHelper { } /// Generate a cofiguration augmented with options from the environment - pub fn build(mut self) -> DeltaResult> { + pub fn build(mut self) -> Result> { let mut has_credential = false; if self.config.contains_key(&AzureConfigKey::UseAzureCli) { diff --git a/crates/azure/src/error.rs b/crates/azure/src/error.rs new file mode 100644 index 0000000000..acc18f67f9 --- /dev/null +++ b/crates/azure/src/error.rs @@ -0,0 +1,22 @@ +use deltalake_core::errors::DeltaTableError; + +pub(crate) type Result = std::result::Result; + +#[derive(thiserror::Error, Debug)] +pub(crate) enum Error { + #[allow(dead_code)] + #[error("failed to parse config: {0}")] + Parse(String), + + #[error(transparent)] + ObjectStore(#[from] object_store::Error), +} + +impl From for DeltaTableError { + fn from(e: Error) -> Self { + match e { + Error::Parse(msg) => DeltaTableError::Generic(msg), + Error::ObjectStore(e) => DeltaTableError::ObjectStore { source: e }, + } + } +} diff --git a/crates/azure/src/lib.rs b/crates/azure/src/lib.rs new file mode 100644 index 0000000000..9b957c7b5e --- /dev/null +++ b/crates/azure/src/lib.rs @@ -0,0 +1,69 @@ +use std::collections::HashMap; +use std::str::FromStr; +use std::sync::Arc; + +use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; +use deltalake_core::storage::{ + factories, url_prefix_handler, ObjectStoreFactory, ObjectStoreRef, StorageOptions, +}; +use deltalake_core::{DeltaResult, Path}; +use object_store::azure::AzureConfigKey; +use object_store::parse_url_opts; +use url::Url; + +mod config; +pub mod error; + +trait AzureOptions { + fn as_azure_options(&self) -> HashMap; +} + +impl AzureOptions for StorageOptions { + fn as_azure_options(&self) -> HashMap { + self.0 + .iter() + .filter_map(|(key, value)| { + Some(( + AzureConfigKey::from_str(&key.to_ascii_lowercase()).ok()?, + value.clone(), + )) + }) + .collect() + } +} + +#[derive(Clone, Default, Debug)] +pub struct AzureFactory {} + +impl ObjectStoreFactory for AzureFactory { + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + let config = config::AzureConfigHelper::try_new(options.as_azure_options())?.build()?; + let (store, prefix) = parse_url_opts(url, config)?; + Ok((url_prefix_handler(store, prefix.clone())?, prefix)) + } +} + +impl LogStoreFactory for AzureFactory { + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + Ok(default_logstore(store, location, options)) + } +} + +/// Register an [ObjectStoreFactory] for common Azure [Url] schemes +pub fn register_handlers(_additional_prefixes: Option) { + let factory = Arc::new(AzureFactory {}); + for scheme in ["az", "adl", "azure", "abfs", "abfss"].iter() { + let url = Url::parse(&format!("{}://", scheme)).unwrap(); + factories().insert(url.clone(), factory.clone()); + logstores().insert(url.clone(), factory.clone()); + } +} diff --git a/crates/azure/tests/context.rs b/crates/azure/tests/context.rs new file mode 100644 index 0000000000..1f622191ad --- /dev/null +++ b/crates/azure/tests/context.rs @@ -0,0 +1,161 @@ +use chrono::Utc; +use deltalake_azure::register_handlers; +use deltalake_test::utils::*; +use std::process::ExitStatus; + +/// Kinds of storage integration +#[derive(Clone, Debug)] +pub enum MsftIntegration { + Azure(String), + Onelake, + OnelakeAbfs, +} + +impl Default for MsftIntegration { + fn default() -> Self { + register_handlers(None); + Self::Azure(format!("test-delta-table-{}", Utc::now().timestamp())) + } +} + +impl StorageIntegration for MsftIntegration { + fn prepare_env(&self) { + match self { + Self::Azure(_) => az_cli::prepare_env(), + Self::Onelake => onelake_cli::prepare_env(), + Self::OnelakeAbfs => onelake_cli::prepare_env(), + } + } + + fn create_bucket(&self) -> std::io::Result { + match self { + Self::Azure(_) => az_cli::create_container(self.bucket_name()), + Self::Onelake => Ok(ExitStatus::default()), + Self::OnelakeAbfs => Ok(ExitStatus::default()), + } + } + + fn bucket_name(&self) -> String { + match self { + Self::Azure(name) => name.clone(), + Self::Onelake => { + let account_name = + std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap_or(String::from("onelake")); + let container_name = std::env::var("AZURE_STORAGE_CONTAINER_NAME") + .unwrap_or(String::from("delta-rs")); + format!( + "{0}.dfs.fabric.microsoft.com/{1}", + account_name, container_name + ) + } + Self::OnelakeAbfs => { + let account_name = + std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap_or(String::from("onelake")); + let container_name = std::env::var("AZURE_STORAGE_CONTAINER_NAME") + .unwrap_or(String::from("delta-rs")); + format!( + "{0}@{1}.dfs.fabric.microsoft.com", + container_name, account_name + ) + } + } + } + + fn root_uri(&self) -> String { + format!("az://{}", self.bucket_name()) + } + + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + let destination = format!("{}/{}", self.bucket_name(), destination); + az_cli::copy_directory(source, destination) + } +} + +impl Drop for MsftIntegration { + fn drop(&mut self) { + az_cli::delete_container(self.bucket_name()).expect("Failed to drop bucket"); + } +} + +//cli for onelake +mod onelake_cli { + use super::set_env_if_not_set; + /// prepare_env + pub fn prepare_env() { + let token = "jwt-token"; + set_env_if_not_set("AZURE_STORAGE_USE_EMULATOR", "0"); + set_env_if_not_set("AZURE_STORAGE_ACCOUNT_NAME", "daily-onelake"); + set_env_if_not_set( + "AZURE_STORAGE_CONTAINER_NAME", + "86bc63cf-5086-42e0-b16d-6bc580d1dc87", + ); + set_env_if_not_set("AZURE_STORAGE_TOKEN", token); + } +} + +/// small wrapper around az cli +mod az_cli { + use super::set_env_if_not_set; + use std::process::{Command, ExitStatus}; + + /// Create a new bucket + pub fn create_container(container_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "container", + "create", + "-n", + container_name.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + /// delete bucket + pub fn delete_container(container_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "container", + "delete", + "-n", + container_name.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + /// copy directory + pub fn copy_directory( + source: impl AsRef, + destination: impl AsRef, + ) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "blob", + "upload-batch", + "-s", + source.as_ref(), + "-d", + destination.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + /// prepare_env + pub fn prepare_env() { + set_env_if_not_set("AZURE_STORAGE_USE_EMULATOR", "1"); + set_env_if_not_set("AZURE_STORAGE_ACCOUNT_NAME", "devstoreaccount1"); + set_env_if_not_set("AZURE_STORAGE_ACCOUNT_KEY", "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="); + set_env_if_not_set( + "AZURE_STORAGE_CONNECTION_STRING", + "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://localhost:10000/devstoreaccount1;" + ); + } +} diff --git a/crates/azure/tests/integration.rs b/crates/azure/tests/integration.rs new file mode 100644 index 0000000000..5230462c92 --- /dev/null +++ b/crates/azure/tests/integration.rs @@ -0,0 +1,88 @@ +#![cfg(feature = "integration_test")] + +use bytes::Bytes; +use deltalake_core::DeltaTableBuilder; +use deltalake_test::read::read_table_paths; +use deltalake_test::{test_concurrent_writes, test_read_tables, IntegrationContext, TestResult}; +use object_store::path::Path; +use serial_test::serial; + +mod context; +use context::*; + +static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; +/// TEST_PREFIXES as they should appear in object stores. +static TEST_PREFIXES_ENCODED: &[&str] = &["my table", "%E4%BD%A0%E5%A5%BD/%F0%9F%98%8A"]; + +#[tokio::test] +#[serial] +async fn test_read_tables_azure() -> TestResult { + let context = IntegrationContext::new(Box::new(MsftIntegration::default()))?; + + test_read_tables(&context).await?; + + for (prefix, _prefix_encoded) in TEST_PREFIXES.iter().zip(TEST_PREFIXES_ENCODED.iter()) { + read_table_paths(&context, prefix, prefix).await?; + } + + Ok(()) +} + +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] +#[serial] +async fn test_concurrency_azure() -> TestResult { + let context = IntegrationContext::new(Box::new(MsftIntegration::default()))?; + + test_concurrent_writes(&context).await?; + + Ok(()) +} + +// NOTE: This test is ignored based on [this +// comment](https://github.com/delta-io/delta-rs/pull/1564#issuecomment-1721048753) and we should +// figure out a way to re-enable this test at least in the GitHub Actions CI environment +#[ignore] +#[cfg(feature = "azure")] +#[tokio::test] +#[serial] +async fn test_object_store_onelake() -> TestResult { + let path = Path::from("17d3977c-d46e-4bae-8fed-ff467e674aed/Files/SampleCustomerList.csv"); + let context = IntegrationContext::new(Box::new(MsftIntegration::Onelake))?; + read_write_test_onelake(&context, &path).await?; + Ok(()) +} + +// NOTE: This test is ignored based on [this +// comment](https://github.com/delta-io/delta-rs/pull/1564#issuecomment-1721048753) and we should +// figure out a way to re-enable this test at least in the GitHub Actions CI environment +#[ignore] +#[cfg(feature = "azure")] +#[tokio::test] +#[serial] +async fn test_object_store_onelake_abfs() -> TestResult { + let path = Path::from("17d3977c-d46e-4bae-8fed-ff467e674aed/Files/SampleCustomerList.csv"); + let context = IntegrationContext::new(Box::new(MsftIntegration::OnelakeAbfs))?; + read_write_test_onelake(&context, &path).await?; + Ok(()) +} + +#[allow(dead_code)] +async fn read_write_test_onelake(context: &IntegrationContext, path: &Path) -> TestResult { + let delta_store = DeltaTableBuilder::from_uri(&context.root_uri()) + .with_allow_http(true) + .build_storage()? + .object_store(); + + let expected = Bytes::from_static(b"test world from delta-rs on friday"); + + delta_store.put(path, expected.clone()).await.unwrap(); + let fetched = delta_store.get(path).await.unwrap().bytes().await.unwrap(); + assert_eq!(expected, fetched); + + for range in [0..10, 3..5, 0..expected.len()] { + let data = delta_store.get_range(path, range.clone()).await.unwrap(); + assert_eq!(&data[..], &expected[range]) + } + + Ok(()) +} diff --git a/crates/benchmarks/Cargo.toml b/crates/benchmarks/Cargo.toml index 76bcc8a312..c7a6248cfb 100644 --- a/crates/benchmarks/Cargo.toml +++ b/crates/benchmarks/Cargo.toml @@ -41,6 +41,6 @@ datafusion-sql = { workspace = true } datafusion-physical-expr = { workspace = true } [dependencies.deltalake-core] -path = "../deltalake-core" +path = "../core" version = "0" features = ["datafusion"] diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index 5afa3e6f35..ea43171052 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -193,10 +193,10 @@ async fn benchmark_merge_tpcds( merge: fn(DataFrame, DeltaTable) -> Result, ) -> Result<(core::time::Duration, MergeMetrics), DataFusionError> { let table = DeltaTableBuilder::from_uri(path).load().await?; - let file_count = table.state.files().len(); + let file_count = table.snapshot()?.files_count(); let provider = DeltaTableProvider::try_new( - table.state.clone(), + table.snapshot()?.clone(), table.log_store(), DeltaScanConfig { file_column_name: Some("file_path".to_string()), @@ -214,7 +214,7 @@ async fn benchmark_merge_tpcds( .filter(col("r").lt_eq(lit(parameters.sample_files)))?; let file_sample = files.collect_partitioned().await?; - let schema = file_sample.get(0).unwrap().get(0).unwrap().schema(); + let schema = file_sample.first().unwrap().first().unwrap().schema(); let mem_table = Arc::new(MemTable::try_new(schema, file_sample)?); ctx.register_table("file_sample", mem_table)?; let file_sample_count = ctx.table("file_sample").await?.count().await?; @@ -265,6 +265,14 @@ async fn benchmark_merge_tpcds( .object_store() .delete(&Path::parse("_delta_log/00000000000000000002.json")?) .await?; + table + .object_store() + .delete(&Path::parse("_delta_log/00000000000000000003.json")?) + .await?; + let _ = table + .object_store() + .delete(&Path::parse("_delta_log/00000000000000000004.json")?) + .await; Ok((duration, metrics)) } @@ -355,6 +363,7 @@ struct MergePrefArgs { #[tokio::main] async fn main() { + type MergeOp = fn(DataFrame, DeltaTable) -> Result; match MergePrefArgs::parse().command { Command::Convert(Convert { tpcds_path, @@ -364,11 +373,9 @@ async fn main() { .await .unwrap(); } + Command::Bench(BenchArg { table_path, name }) => { - let (merge_op, params): ( - fn(DataFrame, DeltaTable) -> Result, - MergePerfParams, - ) = match name { + let (merge_op, params): (MergeOp, MergePerfParams) = match name { MergeBench::Upsert(params) => (merge_upsert, params), MergeBench::Delete(params) => (merge_delete, params), MergeBench::Insert(params) => (merge_insert, params), diff --git a/crates/deltalake-catalog-glue/Cargo.toml b/crates/catalog-glue/Cargo.toml similarity index 51% rename from crates/deltalake-catalog-glue/Cargo.toml rename to crates/catalog-glue/Cargo.toml index aee12726e6..427323c3e0 100644 --- a/crates/deltalake-catalog-glue/Cargo.toml +++ b/crates/catalog-glue/Cargo.toml @@ -1,21 +1,24 @@ [package] name = "deltalake-catalog-glue" version = "0.1.0" -edition = "2021" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true [dependencies] async-trait = { workspace = true } -aws-config = "0.57.1" -aws-sdk-glue = "0.35.0" -deltalake-core = { path = "../deltalake-core" } +aws-config = "1" +aws-sdk-glue = "1" +deltalake-core = { version = "0.17.0", path = "../core" } # This can depend on a lowest common denominator of core once that's released # deltalake_core = { version = "0.17.0" } -log = "0.4" thiserror = { workspace = true } [dev-dependencies] tokio = { version = "1", features = ["macros", "rt-multi-thread"] } - -[features] -default = [] -native-tls = [] diff --git a/crates/deltalake-catalog-glue/Makefile b/crates/catalog-glue/Makefile similarity index 100% rename from crates/deltalake-catalog-glue/Makefile rename to crates/catalog-glue/Makefile diff --git a/crates/deltalake-catalog-glue/examples/demo.rs b/crates/catalog-glue/examples/demo.rs similarity index 100% rename from crates/deltalake-catalog-glue/examples/demo.rs rename to crates/catalog-glue/examples/demo.rs diff --git a/crates/deltalake-catalog-glue/src/lib.rs b/crates/catalog-glue/src/lib.rs similarity index 96% rename from crates/deltalake-catalog-glue/src/lib.rs rename to crates/catalog-glue/src/lib.rs index 32894cc36e..e9ef449be2 100644 --- a/crates/deltalake-catalog-glue/src/lib.rs +++ b/crates/catalog-glue/src/lib.rs @@ -1,8 +1,7 @@ //! Glue Data Catalog. //! -use aws_config::SdkConfig; +use aws_config::{BehaviorVersion, SdkConfig}; use deltalake_core::data_catalog::{DataCatalog, DataCatalogError}; -use log::*; #[derive(thiserror::Error, Debug)] pub enum GlueError { @@ -38,7 +37,7 @@ pub struct GlueDataCatalog { impl GlueDataCatalog { /// Creates a new GlueDataCatalog with environmental configuration pub async fn from_env() -> Result { - let config = aws_config::load_from_env().await; + let config = aws_config::load_defaults(BehaviorVersion::latest()).await; let client = aws_sdk_glue::Client::new(&config); Ok(Self { client }) } diff --git a/crates/core/.gitignore b/crates/core/.gitignore new file mode 100644 index 0000000000..fd7fc6ad51 --- /dev/null +++ b/crates/core/.gitignore @@ -0,0 +1,4 @@ +target/ +/.idea/ +*.bat +tests/data diff --git a/crates/deltalake-core/.ignore b/crates/core/.ignore similarity index 100% rename from crates/deltalake-core/.ignore rename to crates/core/.ignore diff --git a/crates/core/Cargo.toml b/crates/core/Cargo.toml new file mode 100644 index 0000000000..8a2f2d6817 --- /dev/null +++ b/crates/core/Cargo.toml @@ -0,0 +1,129 @@ +[package] +name = "deltalake-core" +version = "0.17.1" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[package.metadata.docs.rs] +features = ["datafusion", "json", "unity-experimental"] + +[dependencies] +# arrow +arrow = { workspace = true } +arrow-arith = { workspace = true } +arrow-array = { workspace = true } +arrow-buffer = { workspace = true } +arrow-cast = { workspace = true } +arrow-ipc = { workspace = true } +arrow-json = { workspace = true } +arrow-ord = { workspace = true } +arrow-row = { workspace = true } +arrow-schema = { workspace = true, features = ["serde"] } +arrow-select = { workspace = true } +parquet = { workspace = true, features = [ + "async", + "object_store", +] } +pin-project-lite = "^0.2.7" + +# datafusion +datafusion = { workspace = true, optional = true } +datafusion-expr = { workspace = true, optional = true } +datafusion-common = { workspace = true, optional = true } +datafusion-proto = { workspace = true, optional = true } +datafusion-sql = { workspace = true, optional = true } +datafusion-physical-expr = { workspace = true, optional = true } + +# serde +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true } + +# "stdlib" +bytes = { workspace = true } +chrono = { workspace = true, default-features = false, features = ["clock"] } +hashbrown = "0.14.3" +regex = { workspace = true } +thiserror = { workspace = true } +uuid = { workspace = true, features = ["serde", "v4"] } +url = { workspace = true } + +# runtime +async-trait = { workspace = true } +futures = { workspace = true } +num_cpus = { workspace = true } +tokio = { workspace = true, features = [ + "macros", + "rt", + "rt-multi-thread", + "sync", + "fs", + "parking_lot", +] } + +# other deps (these should be organized and pulled into workspace.dependencies as necessary) +cfg-if = "1" +dashmap = "5" +errno = "0.3" +either = "1.8" +fix-hidden-lifetime-bug = "0.2" +hyper = { version = "0.14", optional = true } +indexmap = "2.2.1" +itertools = "0.12" +lazy_static = "1" +libc = ">=0.2.90, <1" +num-bigint = "0.4" +num-traits = "0.2.15" +object_store = { workspace = true } +once_cell = "1.16.0" +parking_lot = "0.12" +percent-encoding = "2" +roaring = "0.10.1" +tracing = { workspace = true } +rand = "0.8" +z85 = "3.0.5" +maplit = "1" + +# Unity +reqwest = { version = "0.11.18", default-features = false, features = [ + "rustls-tls", + "json", +], optional = true } +sqlparser = { version = "0.41", optional = true } + +[dev-dependencies] +criterion = "0.5" +ctor = "0" +deltalake-test = { path = "../test", features = ["datafusion"] } +dotenvy = "0" +hyper = { version = "0.14", features = ["server"] } +maplit = "1" +pretty_assertions = "1.2.1" +pretty_env_logger = "0.5.0" +rand = "0.8" +serial_test = "3" +tempfile = "3" +tokio = { version = "1", features = ["macros", "rt-multi-thread"] } +utime = "0.3" + +[features] +default = [] +datafusion = [ + "dep:datafusion", + "datafusion-expr", + "datafusion-common", + "datafusion-proto", + "datafusion-physical-expr", + "datafusion-sql", + "sqlparser", +] +datafusion-ext = ["datafusion"] +json = ["parquet/json"] +python = ["arrow/pyarrow"] +unity-experimental = ["reqwest", "hyper"] diff --git a/crates/deltalake-core/README.md b/crates/core/README.md similarity index 82% rename from crates/deltalake-core/README.md rename to crates/core/README.md index 64d17dcae9..7cb674ea11 100644 --- a/crates/deltalake-core/README.md +++ b/crates/core/README.md @@ -48,11 +48,7 @@ cargo run --example read_delta_table - `datafusion` - enable the `datafusion::datasource::TableProvider` trait implementation for Delta Tables, allowing them to be queried using [DataFusion](https://github.com/apache/arrow-datafusion). - `datafusion-ext` - DEPRECATED: alias for `datafusion` feature - `gcs` - enable the Google storage backend to work with Delta Tables in Google Cloud Storage. -- `hdfs` - enable the HDFS storage backend to work with Delta Tables in HDFS. - `json` - enable the JSON feature of the `parquet` crate for better JSON interoperability. -- `parquet2` - use parquet2 for checkpoint deserialization. Since `arrow` and `parquet` features are enabled by default for backwards compatibility, this feature needs to be used with `--no-default-features`. -- `s3` - enable the S3 storage backend to work with Delta Tables in AWS S3. Uses [rustls](https://github.com/ctz/rustls). -- `s3-native-tls` - enable the S3 storage backend but rely on OpenSSL. ## Development diff --git a/crates/deltalake-core/src/data_catalog/client/backoff.rs b/crates/core/src/data_catalog/client/backoff.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/client/backoff.rs rename to crates/core/src/data_catalog/client/backoff.rs diff --git a/crates/deltalake-core/src/data_catalog/client/mock_server.rs b/crates/core/src/data_catalog/client/mock_server.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/client/mock_server.rs rename to crates/core/src/data_catalog/client/mock_server.rs diff --git a/crates/deltalake-core/src/data_catalog/client/mod.rs b/crates/core/src/data_catalog/client/mod.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/client/mod.rs rename to crates/core/src/data_catalog/client/mod.rs diff --git a/crates/deltalake-core/src/data_catalog/client/pagination.rs b/crates/core/src/data_catalog/client/pagination.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/client/pagination.rs rename to crates/core/src/data_catalog/client/pagination.rs diff --git a/crates/deltalake-core/src/data_catalog/client/retry.rs b/crates/core/src/data_catalog/client/retry.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/client/retry.rs rename to crates/core/src/data_catalog/client/retry.rs diff --git a/crates/deltalake-core/src/data_catalog/client/token.rs b/crates/core/src/data_catalog/client/token.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/client/token.rs rename to crates/core/src/data_catalog/client/token.rs diff --git a/crates/deltalake-core/src/data_catalog/mod.rs b/crates/core/src/data_catalog/mod.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/mod.rs rename to crates/core/src/data_catalog/mod.rs diff --git a/crates/deltalake-core/src/data_catalog/storage/mod.rs b/crates/core/src/data_catalog/storage/mod.rs similarity index 93% rename from crates/deltalake-core/src/data_catalog/storage/mod.rs rename to crates/core/src/data_catalog/storage/mod.rs index 729b5de224..5a25054316 100644 --- a/crates/deltalake-core/src/data_catalog/storage/mod.rs +++ b/crates/core/src/data_catalog/storage/mod.rs @@ -14,7 +14,7 @@ use object_store::ObjectStore; use crate::errors::DeltaResult; use crate::open_table_with_storage_options; -use crate::storage::config::{configure_store, StorageOptions}; +use crate::storage::*; use crate::table::builder::ensure_table_uri; const DELTA_LOG_FOLDER: &str = "_delta_log"; @@ -47,9 +47,9 @@ impl ListingSchemaProvider { storage_options: Option>, ) -> DeltaResult { let uri = ensure_table_uri(root_uri)?; - let mut storage_options = storage_options.unwrap_or_default().into(); + let storage_options = storage_options.unwrap_or_default().into(); // We already parsed the url, so unwrapping is safe. - let store = configure_store(&uri, &mut storage_options)?; + let store = store_for(&uri)?; Ok(Self { authority: uri.to_string(), store, @@ -60,7 +60,7 @@ impl ListingSchemaProvider { /// Reload table information from ObjectStore pub async fn refresh(&self) -> datafusion_common::Result<()> { - let entries: Vec<_> = self.store.list(None).await?.try_collect().await?; + let entries: Vec<_> = self.store.list(None).try_collect().await?; let mut tables = HashSet::new(); for file in entries.iter() { let mut parent = Path::new(file.location.as_ref()); @@ -163,7 +163,7 @@ mod tests { #[tokio::test] async fn test_table_names() { - let fs = ListingSchemaProvider::try_new("./tests/data/", None).unwrap(); + let fs = ListingSchemaProvider::try_new("../test/tests/data/", None).unwrap(); fs.refresh().await.unwrap(); let table_names = fs.table_names(); assert!(table_names.len() > 20); @@ -172,7 +172,7 @@ mod tests { #[tokio::test] async fn test_query_table() { - let schema = Arc::new(ListingSchemaProvider::try_new("./tests/data/", None).unwrap()); + let schema = Arc::new(ListingSchemaProvider::try_new("../test/tests/data/", None).unwrap()); schema.refresh().await.unwrap(); let ctx = SessionContext::new(); diff --git a/crates/deltalake-core/src/data_catalog/unity/credential.rs b/crates/core/src/data_catalog/unity/credential.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/unity/credential.rs rename to crates/core/src/data_catalog/unity/credential.rs diff --git a/crates/deltalake-core/src/data_catalog/unity/datafusion.rs b/crates/core/src/data_catalog/unity/datafusion.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/unity/datafusion.rs rename to crates/core/src/data_catalog/unity/datafusion.rs diff --git a/crates/deltalake-core/src/data_catalog/unity/mod.rs b/crates/core/src/data_catalog/unity/mod.rs similarity index 100% rename from crates/deltalake-core/src/data_catalog/unity/mod.rs rename to crates/core/src/data_catalog/unity/mod.rs diff --git a/crates/deltalake-core/src/data_catalog/unity/models.rs b/crates/core/src/data_catalog/unity/models.rs similarity index 93% rename from crates/deltalake-core/src/data_catalog/unity/models.rs rename to crates/core/src/data_catalog/unity/models.rs index 959eafa0a0..e1c8b7d1b7 100644 --- a/crates/deltalake-core/src/data_catalog/unity/models.rs +++ b/crates/core/src/data_catalog/unity/models.rs @@ -66,6 +66,7 @@ pub enum ListTableSummariesResponse { /// Successful response Success { /// Basic table infos + #[serde(default)] tables: Vec, /// Continuation token next_page_token: Option, @@ -441,6 +442,16 @@ pub(crate) mod tests { } "#; + pub(crate) const LIST_TABLES: &str = r#" + { + "tables": [{ + "full_name": "catalog.schema.table_name", + "table_type": "MANAGED" + }] + } + "#; + pub(crate) const LIST_TABLES_EMPTY: &str = "{}"; + #[test] fn test_responses() { let list_schemas: Result = @@ -458,6 +469,21 @@ pub(crate) mod tests { GetTableResponse::Success { .. } )); + let list_tables: Result = serde_json::from_str(LIST_TABLES); + assert!(list_tables.is_ok()); + assert!(matches!( + list_tables.unwrap(), + ListTableSummariesResponse::Success { .. } + )); + + let list_tables: Result = + serde_json::from_str(LIST_TABLES_EMPTY); + assert!(list_tables.is_ok()); + assert!(matches!( + list_tables.unwrap(), + ListTableSummariesResponse::Success { .. } + )); + let get_schema: Result = serde_json::from_str(GET_SCHEMA_RESPONSE); assert!(get_schema.is_ok()); assert!(matches!(get_schema.unwrap(), GetSchemaResponse::Success(_))) diff --git a/crates/deltalake-core/src/delta_datafusion/expr.rs b/crates/core/src/delta_datafusion/expr.rs similarity index 84% rename from crates/deltalake-core/src/delta_datafusion/expr.rs rename to crates/core/src/delta_datafusion/expr.rs index e0c284df92..dfe234ad46 100644 --- a/crates/deltalake-core/src/delta_datafusion/expr.rs +++ b/crates/core/src/delta_datafusion/expr.rs @@ -31,8 +31,7 @@ use datafusion::execution::context::SessionState; use datafusion_common::Result as DFResult; use datafusion_common::{config::ConfigOptions, DFSchema, Result, ScalarValue, TableReference}; use datafusion_expr::{ - expr::{InList, ScalarUDF}, - AggregateUDF, Between, BinaryExpr, Cast, Expr, Like, TableSource, + expr::InList, AggregateUDF, Between, BinaryExpr, Cast, Expr, GetIndexedField, Like, TableSource, }; use datafusion_sql::planner::{ContextProvider, SqlToRel}; use sqlparser::ast::escape_quoted_string; @@ -42,6 +41,8 @@ use sqlparser::tokenizer::Tokenizer; use crate::{DeltaResult, DeltaTableError}; +use super::DeltaParserOptions; + pub(crate) struct DeltaContextProvider<'a> { state: &'a SessionState, } @@ -97,7 +98,8 @@ pub(crate) fn parse_predicate_expression( })?; let context_provider = DeltaContextProvider { state: df_state }; - let sql_to_rel = SqlToRel::new(&context_provider); + let sql_to_rel = + SqlToRel::new_with_options(&context_provider, DeltaParserOptions::default().into()); Ok(sql_to_rel.sql_to_expr(sql, schema, &mut Default::default())?) } @@ -183,8 +185,7 @@ impl<'a> Display for SqlFormat<'a> { Expr::IsNotFalse(expr) => write!(f, "{} IS NOT FALSE", SqlFormat { expr }), Expr::IsNotUnknown(expr) => write!(f, "{} IS NOT UNKNOWN", SqlFormat { expr }), Expr::BinaryExpr(expr) => write!(f, "{}", BinaryExprFormat { expr }), - Expr::ScalarFunction(func) => fmt_function(f, &func.fun.to_string(), false, &func.args), - Expr::ScalarUDF(ScalarUDF { fun, args }) => fmt_function(f, &fun.name, false, args), + Expr::ScalarFunction(func) => fmt_function(f, func.func_def.name(), false, &func.args), Expr::Cast(Cast { expr, data_type }) => { write!(f, "arrow_cast({}, '{}')", SqlFormat { expr }, data_type) } @@ -262,6 +263,28 @@ impl<'a> Display for SqlFormat<'a> { write!(f, "{expr} IN ({})", expr_vec_fmt!(list)) } } + Expr::GetIndexedField(GetIndexedField { expr, field }) => match field { + datafusion_expr::GetFieldAccess::NamedStructField { name } => { + write!( + f, + "{}[{}]", + SqlFormat { expr }, + ScalarValueFormat { scalar: name } + ) + } + datafusion_expr::GetFieldAccess::ListIndex { key } => { + write!(f, "{}[{}]", SqlFormat { expr }, SqlFormat { expr: key }) + } + datafusion_expr::GetFieldAccess::ListRange { start, stop } => { + write!( + f, + "{}[{}:{}]", + SqlFormat { expr }, + SqlFormat { expr: start }, + SqlFormat { expr: stop } + ) + } + }, _ => Err(fmt::Error), } } @@ -344,10 +367,11 @@ impl<'a> fmt::Display for ScalarValueFormat<'a> { mod test { use arrow_schema::DataType as ArrowDataType; use datafusion::prelude::SessionContext; - use datafusion_common::{DFSchema, ScalarValue}; - use datafusion_expr::{col, decode, lit, substring, Cast, Expr, ExprSchemable}; + use datafusion_common::{Column, DFSchema, ScalarValue}; + use datafusion_expr::{cardinality, col, decode, lit, substring, Cast, Expr, ExprSchemable}; - use crate::kernel::{DataType, PrimitiveType, StructField, StructType}; + use crate::delta_datafusion::{DataFusionMixins, DeltaSessionContext}; + use crate::kernel::{ArrayType, DataType, PrimitiveType, StructField, StructType}; use crate::{DeltaOps, DeltaTable}; use super::fmt_expr_to_sql; @@ -385,6 +409,11 @@ mod test { DataType::Primitive(PrimitiveType::Integer), true, ), + StructField::new( + "Value3".to_string(), + DataType::Primitive(PrimitiveType::Integer), + true, + ), StructField::new( "modified".to_string(), DataType::Primitive(PrimitiveType::String), @@ -415,6 +444,30 @@ mod test { DataType::Primitive(PrimitiveType::Binary), true, ), + StructField::new( + "_struct".to_string(), + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("a", DataType::Primitive(PrimitiveType::Integer), true), + StructField::new( + "nested", + DataType::Struct(Box::new(StructType::new(vec![StructField::new( + "b", + DataType::Primitive(PrimitiveType::Integer), + true, + )]))), + true, + ), + ]))), + true, + ), + StructField::new( + "_list".to_string(), + DataType::Array(Box::new(ArrayType::new( + DataType::Primitive(PrimitiveType::Integer), + true, + ))), + true, + ), ]); let table = DeltaOps::new_in_memory() @@ -439,7 +492,10 @@ mod test { }), "arrow_cast(1, 'Int32')".to_string() ), - simple!(col("value").eq(lit(3_i64)), "value = 3".to_string()), + simple!( + Expr::Column(Column::from_qualified_name_ignore_case("Value3")).eq(lit(3_i64)), + "Value3 = 3".to_string() + ), simple!(col("active").is_true(), "active IS TRUE".to_string()), simple!(col("active"), "active".to_string()), simple!(col("active").eq(lit(true)), "active = true".to_string()), @@ -519,7 +575,8 @@ mod test { .cast_to::( &arrow_schema::DataType::Utf8, &table - .state + .snapshot() + .unwrap() .input_schema() .unwrap() .as_ref() @@ -531,16 +588,33 @@ mod test { .eq(lit("1")), "arrow_cast(value, 'Utf8') = '1'".to_string() ), + simple!( + col("_struct").field("a").eq(lit(20_i64)), + "_struct['a'] = 20".to_string() + ), + simple!( + col("_struct").field("nested").field("b").eq(lit(20_i64)), + "_struct['nested']['b'] = 20".to_string() + ), + simple!( + col("_list").index(lit(1_i64)).eq(lit(20_i64)), + "_list[1] = 20".to_string() + ), + simple!( + cardinality(col("_list").range(col("value"), lit(10_i64))), + "cardinality(_list[value:10])".to_string() + ), ]; - let session = SessionContext::new(); + let session: SessionContext = DeltaSessionContext::default().into(); for test in tests { let actual = fmt_expr_to_sql(&test.expr).unwrap(); assert_eq!(test.expected, actual); let actual_expr = table - .state + .snapshot() + .unwrap() .parse_predicate_expression(actual, &session.state()) .unwrap(); diff --git a/crates/deltalake-core/src/delta_datafusion/logical.rs b/crates/core/src/delta_datafusion/logical.rs similarity index 75% rename from crates/deltalake-core/src/delta_datafusion/logical.rs rename to crates/core/src/delta_datafusion/logical.rs index 7b05dd57d9..75ed53d1b1 100644 --- a/crates/deltalake-core/src/delta_datafusion/logical.rs +++ b/crates/core/src/delta_datafusion/logical.rs @@ -1,5 +1,7 @@ //! Logical Operations for DataFusion +use std::collections::HashSet; + use datafusion_expr::{LogicalPlan, UserDefinedLogicalNodeCore}; // Metric Observer is used to update DataFusion metrics from a record batch. @@ -10,6 +12,7 @@ pub(crate) struct MetricObserver { // id is preserved during conversion to physical node pub id: String, pub input: LogicalPlan, + pub enable_pushdown: bool, } impl UserDefinedLogicalNodeCore for MetricObserver { @@ -35,6 +38,18 @@ impl UserDefinedLogicalNodeCore for MetricObserver { write!(f, "MetricObserver id={}", &self.id) } + fn prevent_predicate_push_down_columns(&self) -> HashSet { + if self.enable_pushdown { + HashSet::new() + } else { + self.schema() + .fields() + .iter() + .map(|f| f.name().clone()) + .collect() + } + } + fn from_template( &self, _exprs: &[datafusion_expr::Expr], @@ -43,6 +58,7 @@ impl UserDefinedLogicalNodeCore for MetricObserver { MetricObserver { id: self.id.clone(), input: inputs[0].clone(), + enable_pushdown: self.enable_pushdown, } } } diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs similarity index 71% rename from crates/deltalake-core/src/delta_datafusion/mod.rs rename to crates/core/src/delta_datafusion/mod.rs index 973d575904..bcefcb55b8 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -25,14 +25,18 @@ use std::collections::{HashMap, HashSet}; use std::fmt::{self, Debug}; use std::sync::Arc; -use arrow::array::ArrayRef; use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::DataType; -use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, SchemaRef, TimeUnit}; +use arrow::datatypes::{ + DataType as ArrowDataType, Schema as ArrowSchema, SchemaRef, SchemaRef as ArrowSchemaRef, + TimeUnit, +}; 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, TypedDictionaryArray}; +use arrow_cast::display::array_value_to_string; + use arrow_schema::Field; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; @@ -42,40 +46,46 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::datasource::provider::TableProviderFactory; use datafusion::datasource::{listing::PartitionedFile, MemTable, TableProvider, TableType}; -use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; +use datafusion::execution::context::{SessionConfig, SessionContext, SessionState, TaskContext}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; -use datafusion::optimizer::utils::conjunction; use datafusion::physical_expr::PhysicalSortExpr; -use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::limit::LocalLimitExec; use datafusion::physical_plan::{ - ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, + Statistics, }; use datafusion_common::scalar::ScalarValue; -use datafusion_common::stats::Precision; use datafusion_common::tree_node::{TreeNode, TreeNodeVisitor, VisitRecursion}; -use datafusion_common::{Column, DataFusionError, Result as DataFusionResult, ToDFSchema}; -use datafusion_expr::expr::{ScalarFunction, ScalarUDF}; +use datafusion_common::{ + Column, DFSchema, DataFusionError, Result as DataFusionResult, ToDFSchema, +}; +use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::CreateExternalTable; +use datafusion_expr::utils::conjunction; use datafusion_expr::{col, Expr, Extension, LogicalPlan, TableProviderFilterPushDown, Volatility}; 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 log::error; +use datafusion_sql::planner::ParserOptions; +use either::Either; +use futures::TryStreamExt; + +use itertools::Itertools; use object_store::ObjectMeta; use serde::{Deserialize, Serialize}; use url::Url; +use crate::delta_datafusion::expr::parse_predicate_expression; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Add, DataType as DeltaDataType, Invariant, PrimitiveType}; +use crate::kernel::{Add, DataCheck, EagerSnapshot, Invariant, Snapshot}; 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"; @@ -87,7 +97,7 @@ pub mod physical; impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { match err { - DeltaTableError::Arrow { source } => DataFusionError::ArrowError(source), + DeltaTableError::Arrow { source } => DataFusionError::ArrowError(source, None), DeltaTableError::Io { source } => DataFusionError::IoError(source), DeltaTableError::ObjectStore { source } => DataFusionError::ObjectStore(source), DeltaTableError::Parquet { source } => DataFusionError::ParquetError(source), @@ -99,7 +109,7 @@ impl From for DataFusionError { impl From for DeltaTableError { fn from(err: DataFusionError) -> Self { match err { - DataFusionError::ArrowError(source) => DeltaTableError::Arrow { source }, + DataFusionError::ArrowError(source, _) => DeltaTableError::Arrow { source }, DataFusionError::IoError(source) => DeltaTableError::Io { source }, DataFusionError::ObjectStore(source) => DeltaTableError::ObjectStore { source }, DataFusionError::ParquetError(source) => DeltaTableError::Parquet { source }, @@ -108,236 +118,173 @@ impl From for DeltaTableError { } } -fn get_scalar_value(value: Option<&ColumnValueStat>, field: &Arc) -> Precision { - match value { - Some(ColumnValueStat::Value(value)) => to_correct_scalar_value(value, field.data_type()) - .map(|maybe_scalar| maybe_scalar.map(Precision::Exact).unwrap_or_default()) - .unwrap_or_else(|_| { - error!( - "Unable to parse scalar value of {:?} with type {} for column {}", - value, - field.data_type(), - field.name() - ); - Precision::Absent - }), - _ => Precision::Absent, - } -} - -impl DeltaTableState { - /// Provide table level statistics to Datafusion - pub fn datafusion_table_statistics(&self) -> DataFusionResult { - // Statistics only support primitive types. Any non primitive column will not have their statistics captured - // If column statistics are missing for any add actions then we simply downgrade to Absent. - - let schema = self.arrow_schema()?; - // Downgrade statistics to absent if file metadata is not present. - let mut downgrade = false; - let unknown_stats = Statistics::new_unknown(&schema); - - let files = self.files(); - - // Initalize statistics - let mut table_stats = match files.get(0) { - Some(file) => match file.get_stats() { - Ok(Some(stats)) => { - let mut column_statistics = Vec::with_capacity(schema.fields().size()); - let total_byte_size = Precision::Exact(file.size as usize); - let num_rows = Precision::Exact(stats.num_records as usize); - - for field in schema.fields() { - let null_count = match stats.null_count.get(field.name()) { - Some(ColumnCountStat::Value(x)) => Precision::Exact(*x as usize), - _ => Precision::Absent, - }; - - let max_value = get_scalar_value(stats.max_values.get(field.name()), field); - let min_value = get_scalar_value(stats.min_values.get(field.name()), field); - - column_statistics.push(ColumnStatistics { - null_count, - max_value, - min_value, - distinct_count: Precision::Absent, - }); - } - - Statistics { - total_byte_size, - num_rows, - column_statistics, - } - } - Ok(None) => { - downgrade = true; - let mut stats = unknown_stats.clone(); - stats.total_byte_size = Precision::Exact(file.size as usize); - stats - } - _ => return Ok(unknown_stats), - }, - None => { - // The Table is empty - let mut stats = unknown_stats; - stats.num_rows = Precision::Exact(0); - stats.total_byte_size = Precision::Exact(0); - return Ok(stats); - } - }; - - // Populate the remaining statistics. If file statistics are not present then relevant statistics are downgraded to absent. - for file in &files.as_slice()[1..] { - let byte_size = Precision::Exact(file.size as usize); - table_stats.total_byte_size = table_stats.total_byte_size.add(&byte_size); +/// Convience trait for calling common methods on snapshot heirarchies +pub trait DataFusionMixins { + /// The physical datafusion schema of a table + fn arrow_schema(&self) -> DeltaResult; - if !downgrade { - match file.get_stats() { - Ok(Some(stats)) => { - let num_records = Precision::Exact(stats.num_records as usize); + /// Get the table schema as an [`ArrowSchemaRef`] + fn input_schema(&self) -> DeltaResult; - table_stats.num_rows = table_stats.num_rows.add(&num_records); + /// Parse an expression string into a datafusion [`Expr`] + fn parse_predicate_expression( + &self, + expr: impl AsRef, + df_state: &SessionState, + ) -> DeltaResult; +} - for (idx, field) in schema.fields().iter().enumerate() { - let column_stats = table_stats.column_statistics.get_mut(idx).unwrap(); +impl DataFusionMixins for Snapshot { + fn arrow_schema(&self) -> DeltaResult { + _arrow_schema(self, true) + } - let null_count = match stats.null_count.get(field.name()) { - Some(ColumnCountStat::Value(x)) => Precision::Exact(*x as usize), - _ => Precision::Absent, - }; + fn parse_predicate_expression( + &self, + expr: impl AsRef, + df_state: &SessionState, + ) -> DeltaResult { + let schema = DFSchema::try_from(self.arrow_schema()?.as_ref().to_owned())?; + parse_predicate_expression(&schema, expr, df_state) + } - let max_value = - get_scalar_value(stats.max_values.get(field.name()), field); - let min_value = - get_scalar_value(stats.min_values.get(field.name()), field); + fn input_schema(&self) -> DeltaResult { + _arrow_schema(self, false) + } +} - column_stats.null_count = column_stats.null_count.add(&null_count); - column_stats.max_value = column_stats.max_value.max(&max_value); - column_stats.min_value = column_stats.min_value.min(&min_value); - } - } - Ok(None) => { - downgrade = true; - } - Err(_) => return Ok(unknown_stats), - } - } - } +impl DataFusionMixins for EagerSnapshot { + fn arrow_schema(&self) -> DeltaResult { + self.snapshot().arrow_schema() + } - if downgrade { - table_stats.column_statistics = unknown_stats.column_statistics; - table_stats.num_rows = Precision::Absent; - } + fn parse_predicate_expression( + &self, + expr: impl AsRef, + df_state: &SessionState, + ) -> DeltaResult { + self.snapshot().parse_predicate_expression(expr, df_state) + } - Ok(table_stats) + fn input_schema(&self) -> DeltaResult { + self.snapshot().input_schema() } } -// TODO: Collapse with operations/transaction/state.rs method of same name -fn get_prune_stats(table: &DeltaTable, column: &Column, get_max: bool) -> Option { - let field = table - .get_schema() - .ok() - .map(|s| s.field_with_name(&column.name).ok())??; +impl DataFusionMixins for DeltaTableState { + fn arrow_schema(&self) -> DeltaResult { + self.snapshot.arrow_schema() + } - // See issue 1214. Binary type does not support natural order which is required for Datafusion to prune - if let DeltaDataType::Primitive(PrimitiveType::Binary) = &field.data_type() { - return None; + fn input_schema(&self) -> DeltaResult { + self.snapshot.input_schema() } - let data_type = field.data_type().try_into().ok()?; - let partition_columns = &table.metadata().ok()?.partition_columns; + fn parse_predicate_expression( + &self, + expr: impl AsRef, + df_state: &SessionState, + ) -> DeltaResult { + self.snapshot.parse_predicate_expression(expr, df_state) + } +} - let values = table.get_state().files().iter().map(|add| { - if partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - let value = match value { - Some(v) => serde_json::Value::String(v.to_string()), - None => serde_json::Value::Null, - }; - to_correct_scalar_value(&value, &data_type) - .ok() - .flatten() - .unwrap_or( - get_null_of_arrow_type(&data_type).expect("Could not determine null type"), - ) - } else if let Ok(Some(statistics)) = add.get_stats() { - let values = if get_max { - statistics.max_values - } else { - statistics.min_values - }; +fn _arrow_schema(snapshot: &Snapshot, wrap_partitions: bool) -> DeltaResult { + let meta = snapshot.metadata(); + let fields = meta + .schema()? + .fields() + .iter() + .filter(|f| !meta.partition_columns.contains(&f.name().to_string())) + .map(|f| f.try_into()) + .chain( + meta.schema()? + .fields() + .iter() + .filter(|f| meta.partition_columns.contains(&f.name().to_string())) + .map(|f| { + let field = Field::try_from(f)?; + let corrected = if wrap_partitions { + match field.data_type() { + // Only dictionary-encode types that may be large + // // https://github.com/apache/arrow-datafusion/pull/5545 + DataType::Utf8 + | DataType::LargeUtf8 + | DataType::Binary + | DataType::LargeBinary => { + wrap_partition_type_in_dict(field.data_type().clone()) + } + _ => field.data_type().clone(), + } + } else { + field.data_type().clone() + }; + Ok(field.with_data_type(corrected)) + }), + ) + .collect::, _>>()?; - values - .get(&column.name) - .and_then(|f| { - to_correct_scalar_value(f.as_value()?, &data_type) - .ok() - .flatten() - }) - .unwrap_or( - get_null_of_arrow_type(&data_type).expect("Could not determine null type"), - ) - } else { - // No statistics available - get_null_of_arrow_type(&data_type).expect("Could not determine null type") - } - }); - ScalarValue::iter_to_array(values).ok() + Ok(Arc::new(ArrowSchema::new(fields))) } -impl PruningStatistics for DeltaTable { - /// return the minimum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows - fn min_values(&self, column: &Column) -> Option { - get_prune_stats(self, column, false) - } +pub(crate) trait DataFusionFileMixins { + /// Iterate over all files in the log matching a predicate + fn files_matching_predicate(&self, filters: &[Expr]) -> DeltaResult>; +} - /// return the maximum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows. - fn max_values(&self, column: &Column) -> Option { - get_prune_stats(self, column, true) +impl DataFusionFileMixins for EagerSnapshot { + fn files_matching_predicate(&self, filters: &[Expr]) -> DeltaResult> { + files_matching_predicate(self, filters) } +} - /// return the number of containers (e.g. row groups) being - /// pruned with these statistics - fn num_containers(&self) -> usize { - self.get_state().files().len() +pub(crate) fn files_matching_predicate<'a>( + snapshot: &'a EagerSnapshot, + filters: &[Expr], +) -> DeltaResult + 'a> { + if let Some(Some(predicate)) = + (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) + { + let expr = logical_expr_to_physical_expr(&predicate, snapshot.arrow_schema()?.as_ref()); + let pruning_predicate = PruningPredicate::try_new(expr, snapshot.arrow_schema()?)?; + Ok(Either::Left( + snapshot + .file_actions()? + .zip(pruning_predicate.prune(snapshot)?) + .filter_map( + |(action, keep_file)| { + if keep_file { + Some(action) + } else { + None + } + }, + ), + )) + } else { + Ok(Either::Right(snapshot.file_actions()?)) } +} - /// return the number of null values for the named column as an - /// `Option`. - /// - /// Note: the returned array must contain `num_containers()` rows. - fn null_counts(&self, column: &Column) -> Option { - let partition_columns = &self.metadata().ok()?.partition_columns; - - let values = self.get_state().files().iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - if partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - match value { - Some(_) => ScalarValue::UInt64(Some(0)), - None => ScalarValue::UInt64(Some(statistics.num_records as u64)), - } - } else { - statistics - .null_count - .get(&column.name) - .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) - .unwrap_or(ScalarValue::UInt64(None)) - } - } else if partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - match value { - Some(_) => ScalarValue::UInt64(Some(0)), - None => ScalarValue::UInt64(None), - } - } else { - ScalarValue::UInt64(None) - } - }); - ScalarValue::iter_to_array(values).ok() +pub(crate) fn get_path_column<'a>( + batch: &'a RecordBatch, + path_column: &str, +) -> DeltaResult> { + let err = || DeltaTableError::Generic("Unable to obtain Delta-rs path column".to_string()); + batch + .column_by_name(path_column) + .unwrap() + .as_any() + .downcast_ref::>() + .ok_or_else(err)? + .downcast_dict::() + .ok_or_else(err) +} + +impl DeltaTableState { + /// Provide table level statistics to Datafusion + pub fn datafusion_table_statistics(&self) -> Option { + self.snapshot.datafusion_table_statistics() } } @@ -349,14 +296,29 @@ pub(crate) fn register_store(store: LogStoreRef, env: Arc) { env.register_object_store(url, store.object_store()); } -pub(crate) fn logical_schema( +/// The logical schema for a Deltatable is different then protocol level schema since partiton columns must appear at the end of the schema. +/// This is to align with how partition are handled at the physical level +pub(crate) fn df_logical_schema( snapshot: &DeltaTableState, scan_config: &DeltaScanConfig, ) -> DeltaResult { let input_schema = snapshot.arrow_schema()?; - let mut fields = Vec::new(); - for field in input_schema.fields.iter() { - fields.push(field.to_owned()); + let table_partition_cols = &snapshot.metadata().partition_columns; + + let mut fields: Vec> = input_schema + .fields() + .iter() + .filter(|f| !table_partition_cols.contains(f.name())) + .cloned() + .collect(); + + for partition_col in table_partition_cols.iter() { + fields.push(Arc::new( + input_schema + .field_with_name(partition_col) + .unwrap() + .to_owned(), + )); } if let Some(file_column_name) = &scan_config.file_column_name { @@ -517,7 +479,7 @@ impl<'a> DeltaScanBuilder<'a> { .await? } }; - let logical_schema = logical_schema(self.snapshot, &config)?; + let logical_schema = df_logical_schema(self.snapshot, &config)?; let logical_schema = if let Some(used_columns) = self.projection { let mut fields = vec![]; @@ -542,7 +504,7 @@ impl<'a> DeltaScanBuilder<'a> { PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; let files_to_prune = pruning_predicate.prune(self.snapshot)?; self.snapshot - .files() + .file_actions()? .iter() .zip(files_to_prune.into_iter()) .filter_map( @@ -556,7 +518,7 @@ impl<'a> DeltaScanBuilder<'a> { ) .collect() } else { - self.snapshot.files().to_owned() + self.snapshot.file_actions()? } } }; @@ -566,11 +528,7 @@ impl<'a> DeltaScanBuilder<'a> { // However we may want to do some additional balancing in case we are far off from the above. let mut file_groups: HashMap, Vec> = HashMap::new(); - let table_partition_cols = &self - .snapshot - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns; + let table_partition_cols = &self.snapshot.metadata().partition_columns; for action in files.iter() { let mut part = partitioned_file_from_action(action, table_partition_cols, &schema); @@ -613,13 +571,7 @@ impl<'a> DeltaScanBuilder<'a> { let stats = self .snapshot .datafusion_table_statistics() - .unwrap_or_else(|e| { - error!( - "Error while computing table statistics. Using unknown statistics. {}", - e - ); - Statistics::new_unknown(&schema) - }); + .unwrap_or(Statistics::new_unknown(&schema)); let scan = ParquetFormat::new() .create_physical_plan( @@ -633,7 +585,6 @@ impl<'a> DeltaScanBuilder<'a> { limit: self.limit, table_partition_cols, output_ordering: vec![], - infinite_source: false, }, logical_filter.as_ref(), ) @@ -648,6 +599,7 @@ impl<'a> DeltaScanBuilder<'a> { } } +// TODO: implement this for Snapshot, not for DeltaTable #[async_trait] impl TableProvider for DeltaTable { fn as_any(&self) -> &dyn Any { @@ -655,7 +607,7 @@ impl TableProvider for DeltaTable { } fn schema(&self) -> Arc { - self.state.arrow_schema().unwrap() + self.snapshot().unwrap().arrow_schema().unwrap() } fn table_type(&self) -> TableType { @@ -680,7 +632,7 @@ impl TableProvider for DeltaTable { register_store(self.log_store(), session.runtime_env().clone()); let filter_expr = conjunction(filters.iter().cloned()); - let scan = DeltaScanBuilder::new(&self.state, self.log_store(), session) + let scan = DeltaScanBuilder::new(self.snapshot()?, self.log_store(), session) .with_projection(projection) .with_limit(limit) .with_filter(filter_expr) @@ -698,7 +650,7 @@ impl TableProvider for DeltaTable { } fn statistics(&self) -> Option { - self.state.datafusion_table_statistics().ok() + self.snapshot().ok()?.datafusion_table_statistics() } } @@ -718,7 +670,7 @@ impl DeltaTableProvider { config: DeltaScanConfig, ) -> DeltaResult { Ok(DeltaTableProvider { - schema: logical_schema(&snapshot, &config)?, + schema: df_logical_schema(&snapshot, &config)?, snapshot, log_store, config, @@ -777,7 +729,7 @@ impl TableProvider for DeltaTableProvider { } fn statistics(&self) -> Option { - self.snapshot.datafusion_table_statistics().ok() + self.snapshot.datafusion_table_statistics() } } @@ -1010,22 +962,90 @@ pub(crate) fn logical_expr_to_physical_expr( ) -> Arc { let df_schema = schema.clone().to_dfschema().unwrap(); let execution_props = ExecutionProps::new(); - create_physical_expr(expr, &df_schema, schema, &execution_props).unwrap() + create_physical_expr(expr, &df_schema, &execution_props).unwrap() +} + +pub(crate) async fn execute_plan_to_batch( + state: &SessionState, + plan: Arc, +) -> DeltaResult { + let data = + futures::future::try_join_all((0..plan.output_partitioning().partition_count()).map(|p| { + let plan_copy = plan.clone(); + let task_context = state.task_ctx().clone(); + async move { + let batch_stream = plan_copy.execute(p, task_context)?; + + let schema = batch_stream.schema(); + + let batches = batch_stream.try_collect::>().await?; + + DataFusionResult::<_>::Ok(arrow::compute::concat_batches(&schema, batches.iter())?) + } + })) + .await?; + + let batch = arrow::compute::concat_batches(&plan.schema(), data.iter())?; + + Ok(batch) } /// 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 no invariants or constraints + pub fn empty() -> Self { + Self { + invariants: vec![], + constraints: vec![], + ctx: DeltaSessionContext::default().into(), + } + } + + /// Create a new DeltaDataChecker with a specified set of invariants + pub fn new_with_invariants(invariants: Vec) -> Self { + Self { + invariants, + constraints: vec![], + ctx: DeltaSessionContext::default().into(), + } + } + + /// Create a new DeltaDataChecker with a specified set of constraints + pub fn new_with_constraints(constraints: Vec) -> Self { + Self { + constraints, + invariants: vec![], + ctx: DeltaSessionContext::default().into(), + } + } + + /// Specify the Datafusion context + pub fn with_session_context(mut self, context: SessionContext) -> Self { + self.ctx = context; + self + } + + /// Add the specified set of constraints to the current DeltaDataChecker's constraints + pub fn with_extra_constraints(mut self, constraints: Vec) -> Self { + self.constraints.extend(constraints); + self + } + /// Create a new DeltaDataChecker - pub fn new(invariants: Vec) -> Self { + pub fn new(snapshot: &DeltaTableState) -> Self { + let invariants = snapshot.schema().get_invariants().unwrap_or_default(); + let constraints = snapshot.table_config().get_constraints(); Self { invariants, - ctx: SessionContext::new(), + constraints, + ctx: DeltaSessionContext::default().into(), } } @@ -1034,45 +1054,58 @@ 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))?; + + // Use a random table name to avoid clashes when running multiple parallel tasks, e.g. when using a partitioned table + let table_name: String = uuid::Uuid::new_v4().to_string(); + self.ctx.register_table(&table_name, 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 `{}` WHERE NOT ({}) LIMIT 1", + check.get_name(), + table_name, + check.get_expression() ); 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 value: String = dfs[0] + .columns() + .iter() + .map(|c| array_value_to_string(c, 0).unwrap_or(String::from("null"))) + .join(", "); + let msg = format!( - "Invariant ({}) violated by value {}", - invariant.invariant_sql, value + "Check or Invariant ({}) violated by value in row: [{}]", + check.get_expression(), + value ); violations.push(msg); } } + self.ctx.deregister_table(&table_name)?; if !violations.is_empty() { Err(DeltaTableError::InvalidData { violations }) } else { @@ -1235,22 +1268,21 @@ impl TreeNodeVisitor for FindFilesExprProperties { | Expr::Case(_) | Expr::Cast(_) | Expr::TryCast(_) => (), - Expr::ScalarFunction(ScalarFunction { fun, .. }) => { - let v = fun.volatility(); - if v > Volatility::Immutable { - self.result = Err(DeltaTableError::Generic(format!( - "Find files predicate contains nondeterministic function {}", - fun - ))); - return Ok(VisitRecursion::Stop); - } - } - Expr::ScalarUDF(ScalarUDF { fun, .. }) => { - let v = fun.signature.volatility; + Expr::ScalarFunction(ScalarFunction { func_def, .. }) => { + let v = match func_def { + datafusion_expr::ScalarFunctionDefinition::BuiltIn(f) => f.volatility(), + datafusion_expr::ScalarFunctionDefinition::UDF(u) => u.signature().volatility, + datafusion_expr::ScalarFunctionDefinition::Name(n) => { + self.result = Err(DeltaTableError::Generic(format!( + "Cannot determine volatility of find files predicate function {n}", + ))); + return Ok(VisitRecursion::Stop); + } + }; if v > Volatility::Immutable { self.result = Err(DeltaTableError::Generic(format!( "Find files predicate contains nondeterministic function {}", - fun.name + func_def.name() ))); return Ok(VisitRecursion::Stop); } @@ -1287,31 +1319,20 @@ fn join_batches_with_add_actions( let mut files = Vec::with_capacity(batches.iter().map(|batch| batch.num_rows()).sum()); for batch in batches { - let array = batch.column_by_name(path_column).ok_or_else(|| { - DeltaTableError::Generic(format!("Unable to find column {}", path_column)) - })?; - - let iter: Box>> = - if dict_array { - let array = array - .as_any() - .downcast_ref::>() - .ok_or(DeltaTableError::Generic(format!( - "Unable to downcast column {}", - path_column - )))? - .downcast_dict::() - .ok_or(DeltaTableError::Generic(format!( - "Unable to downcast column {}", - path_column - )))?; - Box::new(array.into_iter()) - } else { - let array = array.as_any().downcast_ref::().ok_or( - DeltaTableError::Generic(format!("Unable to downcast column {}", path_column)), - )?; - Box::new(array.into_iter()) - }; + let err = || DeltaTableError::Generic("Unable to obtain Delta-rs path column".to_string()); + + let iter: Box>> = if dict_array { + let array = get_path_column(&batch, path_column)?; + Box::new(array.into_iter()) + } else { + let array = batch + .column_by_name(path_column) + .ok_or_else(err)? + .as_any() + .downcast_ref::() + .ok_or_else(err)?; + Box::new(array.into_iter()) + }; for path in iter { let path = path.ok_or(DeltaTableError::Generic(format!( @@ -1340,7 +1361,7 @@ pub(crate) async fn find_files_scan<'a>( expression: Expr, ) -> DeltaResult> { let candidate_map: HashMap = snapshot - .files() + .file_actions()? .iter() .map(|add| (add.path.clone(), add.to_owned())) .collect(); @@ -1351,7 +1372,7 @@ pub(crate) async fn find_files_scan<'a>( } .build(snapshot)?; - let logical_schema = logical_schema(snapshot, &scan_config)?; + let logical_schema = df_logical_schema(snapshot, &scan_config)?; // Identify which columns we need to project let mut used_columns = expression @@ -1377,7 +1398,6 @@ pub(crate) async fn find_files_scan<'a>( let predicate_expr = create_physical_expr( &Expr::IsTrue(Box::new(expression.clone())), &input_dfschema, - &input_schema, state.execution_props(), )?; @@ -1400,7 +1420,7 @@ pub(crate) async fn scan_memory_table( snapshot: &DeltaTableState, predicate: &Expr, ) -> DeltaResult> { - let actions = snapshot.files().to_owned(); + let actions = snapshot.file_actions()?; let batch = snapshot.add_actions_table(true)?; let mut arrays = Vec::new(); @@ -1457,7 +1477,7 @@ pub async fn find_files<'a>( state: &SessionState, predicate: Option, ) -> DeltaResult { - let current_metadata = snapshot.metadata().ok_or(DeltaTableError::NoMetadata)?; + let current_metadata = snapshot.metadata(); match &predicate { Some(predicate) => { @@ -1488,12 +1508,117 @@ pub async fn find_files<'a>( } } None => Ok(FindFiles { - candidates: snapshot.files().to_owned(), + candidates: snapshot.file_actions()?, partition_scan: true, }), } } +/// A wrapper for sql_parser's ParserOptions to capture sane default table defaults +pub struct DeltaParserOptions { + inner: ParserOptions, +} + +impl Default for DeltaParserOptions { + fn default() -> Self { + DeltaParserOptions { + inner: ParserOptions { + enable_ident_normalization: false, + ..ParserOptions::default() + }, + } + } +} + +impl From for ParserOptions { + fn from(value: DeltaParserOptions) -> Self { + value.inner + } +} + +/// A wrapper for Deltafusion's SessionConfig to capture sane default table defaults +pub struct DeltaSessionConfig { + inner: SessionConfig, +} + +impl Default for DeltaSessionConfig { + fn default() -> Self { + DeltaSessionConfig { + inner: SessionConfig::default() + .set_bool("datafusion.sql_parser.enable_ident_normalization", false), + } + } +} + +impl From for SessionConfig { + fn from(value: DeltaSessionConfig) -> Self { + value.inner + } +} + +/// A wrapper for Deltafusion's SessionContext to capture sane default table defaults +pub struct DeltaSessionContext { + inner: SessionContext, +} + +impl Default for DeltaSessionContext { + fn default() -> Self { + DeltaSessionContext { + inner: SessionContext::new_with_config(DeltaSessionConfig::default().into()), + } + } +} + +impl From for SessionContext { + fn from(value: DeltaSessionContext) -> Self { + value.inner + } +} + +/// A wrapper for Deltafusion's Column to preserve case-sensitivity during string conversion +pub struct DeltaColumn { + inner: Column, +} + +impl From<&str> for DeltaColumn { + fn from(c: &str) -> Self { + DeltaColumn { + inner: Column::from_qualified_name_ignore_case(c), + } + } +} + +/// Create a column, cloning the string +impl From<&String> for DeltaColumn { + fn from(c: &String) -> Self { + DeltaColumn { + inner: Column::from_qualified_name_ignore_case(c), + } + } +} + +/// Create a column, reusing the existing string +impl From for DeltaColumn { + fn from(c: String) -> Self { + DeltaColumn { + inner: Column::from_qualified_name_ignore_case(c), + } + } +} + +impl From for Column { + fn from(value: DeltaColumn) -> Self { + value.inner + } +} + +/// Create a column, resuing the existing datafusion column +impl From for DeltaColumn { + fn from(c: Column) -> Self { + DeltaColumn { inner: c } + } +} + #[cfg(test)] mod tests { use crate::writer::test_utils::get_delta_schema; @@ -1594,7 +1719,6 @@ mod tests { size: 10644, partition_values, modification_time: 1660497727833, - partition_values_parsed: None, data_change: true, stats: None, deletion_vector: None, @@ -1616,7 +1740,8 @@ mod tests { location: Path::from("year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string()), last_modified: Utc.timestamp_millis_opt(1660497727833).unwrap(), size: 10644, - e_tag: None + e_tag: None, + version: None, }, partition_values: [ScalarValue::Int64(Some(2015)), ScalarValue::Int64(Some(1))].to_vec(), range: None, @@ -1641,7 +1766,7 @@ mod tests { .unwrap(); // Empty invariants is okay let invariants: Vec = vec![]; - assert!(DeltaDataChecker::new(invariants) + assert!(DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await .is_ok()); @@ -1651,7 +1776,7 @@ mod tests { Invariant::new("a", "a is not null"), Invariant::new("b", "b < 1000"), ]; - assert!(DeltaDataChecker::new(invariants) + assert!(DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await .is_ok()); @@ -1661,7 +1786,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::new_with_invariants(invariants) + .check_batch(&batch) + .await; assert!(result.is_err()); assert!(matches!(result, Err(DeltaTableError::InvalidData { .. }))); if let Err(DeltaTableError::InvalidData { violations }) = result { @@ -1670,7 +1797,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::new_with_invariants(invariants) + .check_batch(&batch) + .await; assert!(result.is_err()); // Nested invariants are unsupported @@ -1684,7 +1813,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::new_with_invariants(invariants) + .check_batch(&batch) + .await; assert!(result.is_err()); assert!(matches!(result, Err(DeltaTableError::Generic { .. }))); } @@ -1700,7 +1831,7 @@ mod tests { ])); let exec_plan = Arc::from(DeltaScan { table_uri: "s3://my_bucket/this/is/some/path".to_string(), - parquet_scan: Arc::from(EmptyExec::new(false, schema.clone())), + parquet_scan: Arc::from(EmptyExec::new(schema.clone())), config: DeltaScanConfig::default(), logical_schema: schema.clone(), }); @@ -1717,16 +1848,18 @@ mod tests { #[tokio::test] async fn delta_table_provider_with_config() { - let table = crate::open_table("tests/data/delta-2.2.0-partitioned-types") + let table = crate::open_table("../test/tests/data/delta-2.2.0-partitioned-types") .await .unwrap(); let config = DeltaScanConfigBuilder::new() .with_file_column_name(&"file_source") - .build(&table.state) + .build(table.snapshot().unwrap()) .unwrap(); let log_store = table.log_store(); - let provider = DeltaTableProvider::try_new(table.state, log_store, config).unwrap(); + let provider = + DeltaTableProvider::try_new(table.snapshot().unwrap().clone(), log_store, config) + .unwrap(); let ctx = SessionContext::new(); ctx.register_table("test", Arc::new(provider)).unwrap(); @@ -1783,13 +1916,25 @@ mod tests { .await .unwrap(); - let config = DeltaScanConfigBuilder::new().build(&table.state).unwrap(); + let config = DeltaScanConfigBuilder::new() + .build(table.snapshot().unwrap()) + .unwrap(); let log_store = table.log_store(); - let provider = DeltaTableProvider::try_new(table.state, log_store, config).unwrap(); + let provider = + DeltaTableProvider::try_new(table.snapshot().unwrap().clone(), log_store, config) + .unwrap(); + let logical_schema = provider.schema(); let ctx = SessionContext::new(); ctx.register_table("test", Arc::new(provider)).unwrap(); + let expected_logical_order = vec!["value", "modified", "id"]; + let actual_order: Vec = logical_schema + .fields() + .iter() + .map(|f| f.name().to_owned()) + .collect(); + let df = ctx.sql("select * from test").await.unwrap(); let actual = df.collect().await.unwrap(); let expected = vec![ @@ -1803,5 +1948,75 @@ mod tests { "+-------+------------+----+", ]; assert_batches_sorted_eq!(&expected, &actual); + assert_eq!(expected_logical_order, actual_order); + } + + #[tokio::test] + async fn delta_scan_case_sensitive() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("moDified", DataType::Utf8, true), + Field::new("ID", DataType::Utf8, true), + Field::new("vaLue", DataType::Int32, true), + ])); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-01", + "2021-02-01", + "2021-02-02", + "2021-02-02", + ])), + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "C", "D"])), + Arc::new(arrow::array::Int32Array::from(vec![1, 10, 20, 100])), + ], + ) + .unwrap(); + // write some data + let table = crate::DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(crate::protocol::SaveMode::Append) + .await + .unwrap(); + + let config = DeltaScanConfigBuilder::new() + .build(table.snapshot().unwrap()) + .unwrap(); + let log = table.log_store(); + + let provider = + DeltaTableProvider::try_new(table.snapshot().unwrap().clone(), log, config).unwrap(); + let ctx: SessionContext = DeltaSessionContext::default().into(); + ctx.register_table("test", Arc::new(provider)).unwrap(); + + let df = ctx + .sql("select ID, moDified, vaLue from test") + .await + .unwrap(); + let actual = df.collect().await.unwrap(); + let expected = vec![ + "+----+------------+-------+", + "| ID | moDified | vaLue |", + "+----+------------+-------+", + "| A | 2021-02-01 | 1 |", + "| B | 2021-02-01 | 10 |", + "| C | 2021-02-02 | 20 |", + "| D | 2021-02-02 | 100 |", + "+----+------------+-------+", + ]; + assert_batches_sorted_eq!(&expected, &actual); + + /* TODO: Datafusion doesn't have any options to prevent case-sensitivity with the col func */ + /* + let df = ctx + .table("test") + .await + .unwrap() + .select(vec![col("ID"), col("moDified"), col("vaLue")]) + .unwrap(); + let actual = df.collect().await.unwrap(); + assert_batches_sorted_eq!(&expected, &actual); + */ } } diff --git a/crates/deltalake-core/src/delta_datafusion/physical.rs b/crates/core/src/delta_datafusion/physical.rs similarity index 100% rename from crates/deltalake-core/src/delta_datafusion/physical.rs rename to crates/core/src/delta_datafusion/physical.rs diff --git a/crates/deltalake-core/src/errors.rs b/crates/core/src/errors.rs similarity index 95% rename from crates/deltalake-core/src/errors.rs rename to crates/core/src/errors.rs index aaa21a4801..6b6b933d7e 100644 --- a/crates/deltalake-core/src/errors.rs +++ b/crates/core/src/errors.rs @@ -1,7 +1,7 @@ //! Exceptions for the deltalake crate use object_store::Error as ObjectStoreError; -use crate::operations::transaction::TransactionError; +use crate::operations::transaction::{CommitBuilderError, TransactionError}; use crate::protocol::ProtocolError; /// A result returned by delta-rs @@ -23,21 +23,14 @@ pub enum DeltaTableError { }, /// Error returned when parsing checkpoint parquet. - #[cfg(any(feature = "parquet", feature = "parquet2"))] #[error("Failed to parse parquet: {}", .source)] Parquet { /// Parquet error details returned when reading the checkpoint failed. - #[cfg(feature = "parquet")] #[from] source: parquet::errors::ParquetError, - /// Parquet error details returned when reading the checkpoint failed. - #[cfg(feature = "parquet2")] - #[from] - source: parquet2::error::Error, }, /// Error returned when converting the schema in Arrow format failed. - #[cfg(feature = "arrow")] #[error("Failed to convert into Arrow schema: {}", .source)] Arrow { /// Arrow error details returned when converting the schema in Arrow format failed @@ -153,6 +146,13 @@ pub enum DeltaTableError { source: std::io::Error, }, + /// Error raised while preparing a commit + #[error("Commit actions are unsound: {source}")] + CommitValidation { + /// The source error + source: CommitBuilderError, + }, + /// Error raised while commititng transaction #[error("Transaction failed: {source}")] Transaction { @@ -214,6 +214,9 @@ pub enum DeltaTableError { #[error("Table metadata is invalid: {0}")] MetadataError(String), + + #[error("Table has not yet been initialized")] + NotInitialized, } impl From for DeltaTableError { @@ -227,7 +230,6 @@ impl From for DeltaTableError { impl From for DeltaTableError { fn from(value: ProtocolError) -> Self { match value { - #[cfg(feature = "arrow")] ProtocolError::Arrow { source } => DeltaTableError::Arrow { source }, ProtocolError::IO { source } => DeltaTableError::Io { source }, ProtocolError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, diff --git a/crates/core/src/kernel/arrow/extract.rs b/crates/core/src/kernel/arrow/extract.rs new file mode 100644 index 0000000000..1a0d2ad301 --- /dev/null +++ b/crates/core/src/kernel/arrow/extract.rs @@ -0,0 +1,167 @@ +//! Utilties to extract columns from a record batch or nested / complex arrays. + +use std::sync::Arc; + +use arrow_array::{ + Array, ArrowNativeTypeOp, ArrowNumericType, BooleanArray, ListArray, MapArray, PrimitiveArray, + RecordBatch, StringArray, StructArray, +}; +use arrow_schema::{ArrowError, DataType}; + +use crate::{DeltaResult, DeltaTableError}; + +/// Trait to extract a column by name from a record batch or nested / complex array. +pub(crate) trait ProvidesColumnByName { + fn column_by_name(&self, name: &str) -> Option<&Arc>; +} + +impl ProvidesColumnByName for RecordBatch { + fn column_by_name(&self, name: &str) -> Option<&Arc> { + self.column_by_name(name) + } +} + +impl ProvidesColumnByName for StructArray { + fn column_by_name(&self, name: &str) -> Option<&Arc> { + self.column_by_name(name) + } +} + +/// Extracts a column by name and casts it to the given type array type `T`. +/// +/// Returns an error if the column does not exist or if the column is not of type `T`. +pub(crate) fn extract_and_cast<'a, T: Array + 'static>( + arr: &'a dyn ProvidesColumnByName, + name: &'a str, +) -> DeltaResult<&'a T> { + extract_and_cast_opt::(arr, name).ok_or(DeltaTableError::Generic(format!( + "missing-column: {}", + name + ))) +} + +/// Extracts a column by name and casts it to the given type array type `T`. +/// +/// Returns `None` if the column does not exist or if the column is not of type `T`. +pub(crate) fn extract_and_cast_opt<'a, T: Array + 'static>( + array: &'a dyn ProvidesColumnByName, + name: &'a str, +) -> Option<&'a T> { + let mut path_steps = name.split('.'); + let first = path_steps.next()?; + extract_column(array, first, &mut path_steps) + .ok()? + .as_any() + .downcast_ref::() +} + +pub(crate) fn extract_column<'a>( + array: &'a dyn ProvidesColumnByName, + path_step: &str, + remaining_path_steps: &mut impl Iterator, +) -> Result<&'a Arc, ArrowError> { + let child = array + .column_by_name(path_step) + .ok_or(ArrowError::SchemaError(format!( + "No such field: {}", + path_step, + )))?; + + if let Some(next_path_step) = remaining_path_steps.next() { + match child.data_type() { + DataType::Map(_, _) => { + // NOTE a map has exatly one child, but we wnat to be agnostic of its name. + // so we case the current array as map, and use the entries accessor. + let maparr = cast_column_as::(path_step, &Some(child))?; + if let Some(next_path) = remaining_path_steps.next() { + extract_column(maparr.entries(), next_path, remaining_path_steps) + } else { + Ok(child) + // if maparr.entries().num_columns() != 2 { + // return Err(ArrowError::SchemaError(format!( + // "Map {} has {} columns, expected 2", + // path_step, + // maparr.entries().num_columns() + // ))); + // } + // if next_path_step == *maparr.entries().column_names().first().unwrap() { + // Ok(maparr.entries().column(0)) + // } else { + // Ok(maparr.entries().column(1)) + // } + } + } + DataType::List(_) => { + let listarr = cast_column_as::(path_step, &Some(child))?; + if let Some(next_path) = remaining_path_steps.next() { + extract_column( + cast_column_as::(next_path_step, &Some(listarr.values()))?, + next_path, + remaining_path_steps, + ) + } else { + Ok(listarr.values()) + } + } + _ => extract_column( + cast_column_as::(path_step, &Some(child))?, + next_path_step, + remaining_path_steps, + ), + } + } else { + Ok(child) + } +} + +fn cast_column_as<'a, T: Array + 'static>( + name: &str, + column: &Option<&'a Arc>, +) -> Result<&'a T, ArrowError> { + column + .ok_or(ArrowError::SchemaError(format!("No such column: {}", name)))? + .as_any() + .downcast_ref::() + .ok_or(ArrowError::SchemaError(format!( + "{} is not of esxpected type.", + name + ))) +} + +#[inline] +pub(crate) fn read_str(arr: &StringArray, idx: usize) -> DeltaResult<&str> { + read_str_opt(arr, idx).ok_or(DeltaTableError::Generic("missing value".into())) +} + +#[inline] +pub(crate) fn read_str_opt(arr: &StringArray, idx: usize) -> Option<&str> { + arr.is_valid(idx).then(|| arr.value(idx)) +} + +#[inline] +pub(crate) fn read_primitive(arr: &PrimitiveArray, idx: usize) -> DeltaResult +where + T: ArrowNumericType, + T::Native: ArrowNativeTypeOp, +{ + read_primitive_opt(arr, idx).ok_or(DeltaTableError::Generic("missing value".into())) +} + +#[inline] +pub(crate) fn read_primitive_opt(arr: &PrimitiveArray, idx: usize) -> Option +where + T: ArrowNumericType, + T::Native: ArrowNativeTypeOp, +{ + arr.is_valid(idx).then(|| arr.value(idx)) +} + +#[inline] +pub(crate) fn read_bool(arr: &BooleanArray, idx: usize) -> DeltaResult { + read_bool_opt(arr, idx).ok_or(DeltaTableError::Generic("missing value".into())) +} + +#[inline] +pub(crate) fn read_bool_opt(arr: &BooleanArray, idx: usize) -> Option { + arr.is_valid(idx).then(|| arr.value(idx)) +} diff --git a/crates/core/src/kernel/arrow/json.rs b/crates/core/src/kernel/arrow/json.rs new file mode 100644 index 0000000000..dcb56d308a --- /dev/null +++ b/crates/core/src/kernel/arrow/json.rs @@ -0,0 +1,150 @@ +//! Utitlies for reading JSON files and handling JSON data. + +use std::io::{BufRead, BufReader, Cursor}; +use std::task::Poll; + +use arrow_array::{new_null_array, Array, RecordBatch, StringArray}; +use arrow_json::{reader::Decoder, ReaderBuilder}; +use arrow_schema::{ArrowError, SchemaRef as ArrowSchemaRef}; +use arrow_select::concat::concat_batches; +use bytes::{Buf, Bytes}; +use futures::{ready, Stream, StreamExt}; +use object_store::Result as ObjectStoreResult; + +use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; + +#[inline] +pub(crate) fn get_reader(data: &[u8]) -> BufReader> { + BufReader::new(Cursor::new(data)) +} + +#[inline] +pub(crate) fn get_decoder( + schema: ArrowSchemaRef, + config: &DeltaTableConfig, +) -> DeltaResult { + Ok(ReaderBuilder::new(schema) + .with_batch_size(config.log_batch_size) + .build_decoder()?) +} + +fn insert_nulls( + batches: &mut Vec, + null_count: usize, + schema: ArrowSchemaRef, +) -> Result<(), ArrowError> { + let columns = schema + .fields + .iter() + .map(|field| new_null_array(field.data_type(), null_count)) + .collect(); + batches.push(RecordBatch::try_new(schema, columns)?); + Ok(()) +} + +/// Parse an array of JSON strings into a record batch. +/// +/// Null values in the input array are preseverd in the output record batch. +pub(crate) fn parse_json( + json_strings: &StringArray, + output_schema: ArrowSchemaRef, + config: &DeltaTableConfig, +) -> DeltaResult { + let mut decoder = ReaderBuilder::new(output_schema.clone()) + .with_batch_size(config.log_batch_size) + .build_decoder()?; + let mut batches = Vec::new(); + + let mut null_count = 0; + let mut value_count = 0; + let mut value_start = 0; + + for it in 0..json_strings.len() { + if json_strings.is_null(it) { + if value_count > 0 { + let slice = json_strings.slice(value_start, value_count); + let batch = decode_reader(&mut decoder, get_reader(slice.value_data())) + .collect::, _>>()?; + batches.extend(batch); + value_count = 0; + } + null_count += 1; + continue; + } + if value_count == 0 { + value_start = it; + } + if null_count > 0 { + insert_nulls(&mut batches, null_count, output_schema.clone())?; + null_count = 0; + } + value_count += 1; + } + + if null_count > 0 { + insert_nulls(&mut batches, null_count, output_schema.clone())?; + } + + if value_count > 0 { + let slice = json_strings.slice(value_start, value_count); + let batch = decode_reader(&mut decoder, get_reader(slice.value_data())) + .collect::, _>>()?; + batches.extend(batch); + } + + Ok(concat_batches(&output_schema, &batches)?) +} + +/// Decode a stream of bytes into a stream of record batches. +pub(crate) fn decode_stream> + Unpin>( + mut decoder: Decoder, + mut input: S, +) -> impl Stream> { + let mut buffered = Bytes::new(); + futures::stream::poll_fn(move |cx| { + loop { + if buffered.is_empty() { + buffered = match ready!(input.poll_next_unpin(cx)) { + Some(Ok(b)) => b, + Some(Err(e)) => return Poll::Ready(Some(Err(e.into()))), + None => break, + }; + } + let decoded = match decoder.decode(buffered.as_ref()) { + Ok(decoded) => decoded, + Err(e) => return Poll::Ready(Some(Err(e.into()))), + }; + let read = buffered.len(); + buffered.advance(decoded); + if decoded != read { + break; + } + } + + Poll::Ready(decoder.flush().map_err(DeltaTableError::from).transpose()) + }) +} + +/// Decode data prvided by a reader into an iterator of record batches. +pub(crate) fn decode_reader<'a, R: BufRead + 'a>( + decoder: &'a mut Decoder, + mut reader: R, +) -> impl Iterator> + '_ { + let mut next = move || { + loop { + let buf = reader.fill_buf()?; + if buf.is_empty() { + break; // Input exhausted + } + let read = buf.len(); + let decoded = decoder.decode(buf)?; + + reader.consume(decoded); + if decoded != read { + break; // Read batch size + } + } + decoder.flush() + }; + std::iter::from_fn(move || next().map_err(DeltaTableError::from).transpose()) +} diff --git a/crates/deltalake-core/src/kernel/arrow/mod.rs b/crates/core/src/kernel/arrow/mod.rs similarity index 90% rename from crates/deltalake-core/src/kernel/arrow/mod.rs rename to crates/core/src/kernel/arrow/mod.rs index 0c89f6ab48..7dbac2854d 100644 --- a/crates/deltalake-core/src/kernel/arrow/mod.rs +++ b/crates/core/src/kernel/arrow/mod.rs @@ -8,12 +8,23 @@ use arrow_schema::{ }; use lazy_static::lazy_static; -use super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; +use super::{ActionType, ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; -pub mod schemas; +pub(crate) mod extract; +pub(crate) mod json; -const MAP_KEYS_NAME: &str = "keys"; -const MAP_VALUES_NAME: &str = "values"; +const MAP_ROOT_DEFAULT: &str = "entries"; +const MAP_KEY_DEFAULT: &str = "key"; +const MAP_VALUE_DEFAULT: &str = "value"; +const LIST_ROOT_DEFAULT: &str = "item"; + +impl TryFrom for ArrowField { + type Error = ArrowError; + + fn try_from(value: ActionType) -> Result { + value.schema_field().try_into() + } +} impl TryFrom<&StructType> for ArrowSchema { type Error = ArrowError; @@ -22,7 +33,7 @@ impl TryFrom<&StructType> for ArrowSchema { let fields = s .fields() .iter() - .map(>::try_from) + .map(TryInto::try_into) .collect::, ArrowError>>()?; Ok(ArrowSchema::new(fields)) @@ -53,11 +64,11 @@ impl TryFrom<&StructField> for ArrowField { impl TryFrom<&ArrayType> for ArrowField { type Error = ArrowError; - fn try_from(a: &ArrayType) -> Result { Ok(ArrowField::new( - "item", + LIST_ROOT_DEFAULT, ArrowDataType::try_from(a.element_type())?, + // TODO check how to handle nullability a.contains_null(), )) } @@ -68,19 +79,24 @@ impl TryFrom<&MapType> for ArrowField { fn try_from(a: &MapType) -> Result { Ok(ArrowField::new( - "entries", + MAP_ROOT_DEFAULT, ArrowDataType::Struct( vec![ - ArrowField::new(MAP_KEYS_NAME, ArrowDataType::try_from(a.key_type())?, false), ArrowField::new( - MAP_VALUES_NAME, + MAP_KEY_DEFAULT, + ArrowDataType::try_from(a.key_type())?, + false, + ), + ArrowField::new( + MAP_VALUE_DEFAULT, ArrowDataType::try_from(a.value_type())?, a.value_contains_null(), ), ] .into(), ), - false, // always non-null + // always non-null + false, )) } } @@ -102,20 +118,10 @@ impl TryFrom<&DataType> for ArrowDataType { PrimitiveType::Boolean => Ok(ArrowDataType::Boolean), PrimitiveType::Binary => Ok(ArrowDataType::Binary), PrimitiveType::Decimal(precision, scale) => { - let precision = u8::try_from(*precision).map_err(|_| { - ArrowError::SchemaError(format!( - "Invalid precision for decimal: {}", - precision - )) - })?; - let scale = i8::try_from(*scale).map_err(|_| { - ArrowError::SchemaError(format!("Invalid scale for decimal: {}", scale)) - })?; - - if precision <= 38 { - Ok(ArrowDataType::Decimal128(precision, scale)) - } else if precision <= 76 { - Ok(ArrowDataType::Decimal256(precision, scale)) + if precision <= &38 { + Ok(ArrowDataType::Decimal128(*precision, *scale)) + } else if precision <= &76 { + Ok(ArrowDataType::Decimal256(*precision, *scale)) } else { Err(ArrowError::SchemaError(format!( "Precision too large to be represented in Arrow: {}", @@ -128,8 +134,11 @@ impl TryFrom<&DataType> for ArrowDataType { // timezone. Stored as 4 bytes integer representing days since 1970-01-01 Ok(ArrowDataType::Date32) } - PrimitiveType::Timestamp => { - // Issue: https://github.com/delta-io/delta/issues/643 + PrimitiveType::Timestamp => Ok(ArrowDataType::Timestamp( + TimeUnit::Microsecond, + Some("UTC".into()), + )), + PrimitiveType::TimestampNtz => { Ok(ArrowDataType::Timestamp(TimeUnit::Microsecond, None)) } } @@ -137,35 +146,12 @@ impl TryFrom<&DataType> for ArrowDataType { DataType::Struct(s) => Ok(ArrowDataType::Struct( s.fields() .iter() - .map(>::try_from) + .map(TryInto::try_into) .collect::, ArrowError>>()? .into(), )), - DataType::Array(a) => Ok(ArrowDataType::List(Arc::new(>::try_from(a)?))), - DataType::Map(m) => Ok(ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new( - MAP_KEYS_NAME, - >::try_from(m.key_type())?, - false, - ), - ArrowField::new( - MAP_VALUES_NAME, - >::try_from(m.value_type())?, - m.value_contains_null(), - ), - ] - .into(), - ), - false, - )), - false, - )), + DataType::Array(a) => Ok(ArrowDataType::List(Arc::new(a.as_ref().try_into()?))), + DataType::Map(m) => Ok(ArrowDataType::Map(Arc::new(m.as_ref().try_into()?), false)), } } } @@ -197,7 +183,7 @@ impl TryFrom<&ArrowField> for StructField { fn try_from(arrow_field: &ArrowField) -> Result { Ok(StructField::new( arrow_field.name().clone(), - arrow_field.data_type().try_into()?, + DataType::try_from(arrow_field.data_type())?, arrow_field.is_nullable(), ) .with_metadata(arrow_field.metadata().iter().map(|(k, v)| (k.clone(), v)))) @@ -218,23 +204,23 @@ impl TryFrom<&ArrowDataType> for DataType { ArrowDataType::UInt64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type ArrowDataType::UInt32 => Ok(DataType::Primitive(PrimitiveType::Integer)), ArrowDataType::UInt16 => Ok(DataType::Primitive(PrimitiveType::Short)), - ArrowDataType::UInt8 => Ok(DataType::Primitive(PrimitiveType::Boolean)), + ArrowDataType::UInt8 => Ok(DataType::Primitive(PrimitiveType::Byte)), ArrowDataType::Float32 => Ok(DataType::Primitive(PrimitiveType::Float)), ArrowDataType::Float64 => Ok(DataType::Primitive(PrimitiveType::Double)), ArrowDataType::Boolean => Ok(DataType::Primitive(PrimitiveType::Boolean)), ArrowDataType::Binary => Ok(DataType::Primitive(PrimitiveType::Binary)), ArrowDataType::FixedSizeBinary(_) => Ok(DataType::Primitive(PrimitiveType::Binary)), ArrowDataType::LargeBinary => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::Decimal128(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( - *p as i32, *s as i32, - ))), - ArrowDataType::Decimal256(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( - *p as i32, *s as i32, - ))), + ArrowDataType::Decimal128(p, s) => { + Ok(DataType::Primitive(PrimitiveType::Decimal(*p, *s))) + } + ArrowDataType::Decimal256(p, s) => { + Ok(DataType::Primitive(PrimitiveType::Decimal(*p, *s))) + } ArrowDataType::Date32 => Ok(DataType::Primitive(PrimitiveType::Date)), ArrowDataType::Date64 => Ok(DataType::Primitive(PrimitiveType::Date)), ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => { - Ok(DataType::Primitive(PrimitiveType::Timestamp)) + Ok(DataType::Primitive(PrimitiveType::TimestampNtz)) } ArrowDataType::Timestamp(TimeUnit::Microsecond, Some(tz)) if tz.eq_ignore_ascii_case("utc") => @@ -288,11 +274,11 @@ macro_rules! arrow_map { stringify!($fieldname), ArrowDataType::Map( Arc::new(ArrowField::new( - "entries", + MAP_ROOT_DEFAULT, ArrowDataType::Struct( vec![ - ArrowField::new("key", ArrowDataType::Utf8, false), - ArrowField::new("value", ArrowDataType::Utf8, true), + ArrowField::new(MAP_KEY_DEFAULT, ArrowDataType::Utf8, false), + ArrowField::new(MAP_VALUE_DEFAULT, ArrowDataType::Utf8, true), ] .into(), ), @@ -308,11 +294,11 @@ macro_rules! arrow_map { stringify!($fieldname), ArrowDataType::Map( Arc::new(ArrowField::new( - "entries", + MAP_ROOT_DEFAULT, ArrowDataType::Struct( vec![ - ArrowField::new("key", ArrowDataType::Utf8, false), - ArrowField::new("value", ArrowDataType::Utf8, false), + ArrowField::new(MAP_KEY_DEFAULT, ArrowDataType::Utf8, false), + ArrowField::new(MAP_VALUE_DEFAULT, ArrowDataType::Utf8, false), ] .into(), ), @@ -581,11 +567,13 @@ fn max_min_schema_for_fields(dest: &mut Vec, f: &ArrowField) { max_min_schema_for_fields(&mut child_dest, f); } - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); + if !child_dest.is_empty() { + dest.push(ArrowField::new( + f.name(), + ArrowDataType::Struct(child_dest.into()), + true, + )); + } } // don't compute min or max for list, map or binary types ArrowDataType::List(_) | ArrowDataType::Map(_, _) | ArrowDataType::Binary => { /* noop */ } @@ -686,7 +674,7 @@ mod tests { let partition_values_parsed = add_field_map.get("partitionValues_parsed").unwrap(); if let ArrowDataType::Struct(fields) = partition_values_parsed.data_type() { assert_eq!(1, fields.len()); - let field = fields.get(0).unwrap().to_owned(); + let field = fields.first().unwrap().to_owned(); assert_eq!( Arc::new(ArrowField::new("pcol", ArrowDataType::Int32, true)), field @@ -708,7 +696,7 @@ mod tests { "minValues" | "maxValues" | "nullCount" => match v.data_type() { ArrowDataType::Struct(fields) => { assert_eq!(1, fields.len()); - let field = fields.get(0).unwrap().to_owned(); + let field = fields.first().unwrap().to_owned(); let data_type = if k == "nullCount" { ArrowDataType::Int64 } else { @@ -778,13 +766,22 @@ mod tests { let decimal_field = DataType::Primitive(PrimitiveType::Decimal(precision, scale)); assert_eq!( >::try_from(&decimal_field).unwrap(), - ArrowDataType::Decimal128(precision as u8, scale as i8) + ArrowDataType::Decimal128(precision, scale) ); } #[test] fn test_arrow_from_delta_timestamp_type() { let timestamp_field = DataType::Primitive(PrimitiveType::Timestamp); + assert_eq!( + >::try_from(×tamp_field).unwrap(), + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())) + ); + } + + #[test] + fn test_arrow_from_delta_timestampntz_type() { + let timestamp_field = DataType::Primitive(PrimitiveType::TimestampNtz); assert_eq!( >::try_from(×tamp_field).unwrap(), ArrowDataType::Timestamp(TimeUnit::Microsecond, None) @@ -792,11 +789,11 @@ mod tests { } #[test] - fn test_delta_from_arrow_timestamp_type() { + fn test_delta_from_arrow_timestamp_type_no_tz() { let timestamp_field = ArrowDataType::Timestamp(TimeUnit::Microsecond, None); assert_eq!( >::try_from(×tamp_field).unwrap(), - DataType::Primitive(PrimitiveType::Timestamp) + DataType::Primitive(PrimitiveType::TimestampNtz) ); } @@ -861,9 +858,9 @@ mod tests { let entry_offsets_buffer = Buffer::from(entry_offsets.to_byte_slice()); let keys_data = StringArray::from_iter_values(keys); - let keys_field = Arc::new(Field::new("keys", ArrowDataType::Utf8, false)); + let keys_field = Arc::new(Field::new("key", ArrowDataType::Utf8, false)); let values_field = Arc::new(Field::new( - "values", + "value", values.data_type().clone(), values.null_count() > 0, )); diff --git a/crates/deltalake-core/src/kernel/error.rs b/crates/core/src/kernel/error.rs similarity index 95% rename from crates/deltalake-core/src/kernel/error.rs rename to crates/core/src/kernel/error.rs index d4110f8f53..853b10e411 100644 --- a/crates/deltalake-core/src/kernel/error.rs +++ b/crates/core/src/kernel/error.rs @@ -1,12 +1,13 @@ //! Error types for Delta Lake operations. +use super::DataType; + /// A specialized [`Result`] type for Delta Lake operations. pub type DeltaResult = std::result::Result; #[derive(thiserror::Error, Debug)] #[allow(missing_docs)] pub enum Error { - #[cfg(feature = "arrow")] #[error("Arrow error: {0}")] Arrow(#[from] arrow_schema::ArrowError), @@ -19,7 +20,6 @@ pub enum Error { source: Box, }, - #[cfg(feature = "parquet")] #[error("Arrow error: {0}")] Parquet(#[from] parquet::errors::ParquetError), @@ -67,6 +67,9 @@ pub enum Error { #[error("Table metadata is invalid: {0}")] MetadataError(String), + + #[error("Failed to parse value '{0}' as '{1}'")] + Parse(String, DataType), } #[cfg(feature = "object_store")] diff --git a/crates/core/src/kernel/expressions/eval.rs b/crates/core/src/kernel/expressions/eval.rs new file mode 100644 index 0000000000..cb6beea3ad --- /dev/null +++ b/crates/core/src/kernel/expressions/eval.rs @@ -0,0 +1,384 @@ +//! Default Expression handler. +//! +//! Expression handling based on arrow-rs compute kernels. + +use std::sync::Arc; + +use arrow_arith::boolean::{and, is_null, not, or}; +use arrow_arith::numeric::{add, div, mul, sub}; +use arrow_array::{ + Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, Datum, Decimal128Array, Float32Array, + Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, RecordBatch, StringArray, + StructArray, TimestampMicrosecondArray, +}; +use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq}; +use arrow_schema::{ArrowError, Field as ArrowField, Schema as ArrowSchema}; +use arrow_select::nullif::nullif; + +use crate::kernel::arrow::extract::extract_column; +use crate::kernel::error::{DeltaResult, Error}; +use crate::kernel::expressions::{scalars::Scalar, Expression}; +use crate::kernel::expressions::{BinaryOperator, UnaryOperator}; +use crate::kernel::{DataType, PrimitiveType, VariadicOperator}; + +fn downcast_to_bool(arr: &dyn Array) -> DeltaResult<&BooleanArray> { + arr.as_any() + .downcast_ref::() + .ok_or(Error::Generic("expected boolean array".to_string())) +} + +fn wrap_comparison_result(arr: BooleanArray) -> ArrayRef { + Arc::new(arr) as Arc +} + +// TODO leverage scalars / Datum + +impl Scalar { + /// Convert scalar to arrow array. + pub fn to_array(&self, num_rows: usize) -> DeltaResult { + use Scalar::*; + let arr: ArrayRef = match self { + Integer(val) => Arc::new(Int32Array::from_value(*val, num_rows)), + Long(val) => Arc::new(Int64Array::from_value(*val, num_rows)), + Short(val) => Arc::new(Int16Array::from_value(*val, num_rows)), + Byte(val) => Arc::new(Int8Array::from_value(*val, num_rows)), + Float(val) => Arc::new(Float32Array::from_value(*val, num_rows)), + Double(val) => Arc::new(Float64Array::from_value(*val, num_rows)), + String(val) => Arc::new(StringArray::from(vec![val.clone(); num_rows])), + Boolean(val) => Arc::new(BooleanArray::from(vec![*val; num_rows])), + Timestamp(val) => { + Arc::new(TimestampMicrosecondArray::from_value(*val, num_rows).with_timezone("UTC")) + } + TimestampNtz(val) => Arc::new(TimestampMicrosecondArray::from_value(*val, num_rows)), + Date(val) => Arc::new(Date32Array::from_value(*val, num_rows)), + Binary(val) => Arc::new(BinaryArray::from(vec![val.as_slice(); num_rows])), + Decimal(val, precision, scale) => Arc::new( + Decimal128Array::from_value(*val, num_rows) + .with_precision_and_scale(*precision, *scale)?, + ), + Null(data_type) => match data_type { + DataType::Primitive(primitive) => match primitive { + PrimitiveType::Byte => Arc::new(Int8Array::new_null(num_rows)), + PrimitiveType::Short => Arc::new(Int16Array::new_null(num_rows)), + PrimitiveType::Integer => Arc::new(Int32Array::new_null(num_rows)), + PrimitiveType::Long => Arc::new(Int64Array::new_null(num_rows)), + PrimitiveType::Float => Arc::new(Float32Array::new_null(num_rows)), + PrimitiveType::Double => Arc::new(Float64Array::new_null(num_rows)), + PrimitiveType::String => Arc::new(StringArray::new_null(num_rows)), + PrimitiveType::Boolean => Arc::new(BooleanArray::new_null(num_rows)), + PrimitiveType::Timestamp => { + Arc::new(TimestampMicrosecondArray::new_null(num_rows).with_timezone("UTC")) + } + PrimitiveType::TimestampNtz => { + Arc::new(TimestampMicrosecondArray::new_null(num_rows)) + } + PrimitiveType::Date => Arc::new(Date32Array::new_null(num_rows)), + PrimitiveType::Binary => Arc::new(BinaryArray::new_null(num_rows)), + PrimitiveType::Decimal(precision, scale) => Arc::new( + Decimal128Array::new_null(num_rows) + .with_precision_and_scale(*precision, *scale) + .unwrap(), + ), + }, + DataType::Array(_) => unimplemented!(), + DataType::Map { .. } => unimplemented!(), + DataType::Struct { .. } => unimplemented!(), + }, + Struct(values, fields) => { + let mut columns = Vec::with_capacity(values.len()); + for val in values { + columns.push(val.to_array(num_rows)?); + } + Arc::new(StructArray::try_new( + fields + .iter() + .map(TryInto::::try_into) + .collect::, _>>()? + .into(), + columns, + None, + )?) + } + }; + Ok(arr) + } +} + +/// evaluate expression +pub(crate) fn evaluate_expression( + expression: &Expression, + batch: &RecordBatch, + result_type: Option<&DataType>, +) -> DeltaResult { + use BinaryOperator::*; + use Expression::*; + + match (expression, result_type) { + (Literal(scalar), _) => Ok(scalar.to_array(batch.num_rows())?), + (Column(name), _) => { + if name.contains('.') { + let mut path = name.split('.'); + // Safety: we know that the first path step exists, because we checked for '.' + let arr = extract_column(batch, path.next().unwrap(), &mut path).cloned()?; + // NOTE: need to assign first so that rust can figure out lifetimes + Ok(arr) + } else { + batch + .column_by_name(name) + .ok_or(Error::MissingColumn(name.clone())) + .cloned() + } + } + (Struct(fields), Some(DataType::Struct(schema))) => { + let output_schema: ArrowSchema = schema.as_ref().try_into()?; + let mut columns = Vec::with_capacity(fields.len()); + for (expr, field) in fields.iter().zip(schema.fields()) { + columns.push(evaluate_expression(expr, batch, Some(field.data_type()))?); + } + Ok(Arc::new(StructArray::try_new( + output_schema.fields().clone(), + columns, + None, + )?)) + } + (Struct(_), _) => Err(Error::Generic( + "Data type is required to evaluate struct expressions".to_string(), + )), + (UnaryOperation { op, expr }, _) => { + let arr = evaluate_expression(expr.as_ref(), batch, None)?; + Ok(match op { + UnaryOperator::Not => Arc::new(not(downcast_to_bool(&arr)?)?), + UnaryOperator::IsNull => Arc::new(is_null(&arr)?), + }) + } + (BinaryOperation { op, left, right }, _) => { + let left_arr = evaluate_expression(left.as_ref(), batch, None)?; + let right_arr = evaluate_expression(right.as_ref(), batch, None)?; + + type Operation = fn(&dyn Datum, &dyn Datum) -> Result, ArrowError>; + let eval: Operation = match op { + Plus => add, + Minus => sub, + Multiply => mul, + Divide => div, + LessThan => |l, r| lt(l, r).map(wrap_comparison_result), + LessThanOrEqual => |l, r| lt_eq(l, r).map(wrap_comparison_result), + GreaterThan => |l, r| gt(l, r).map(wrap_comparison_result), + GreaterThanOrEqual => |l, r| gt_eq(l, r).map(wrap_comparison_result), + Equal => |l, r| eq(l, r).map(wrap_comparison_result), + NotEqual => |l, r| neq(l, r).map(wrap_comparison_result), + }; + + eval(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + }) + } + (VariadicOperation { op, exprs }, _) => { + let reducer = match op { + VariadicOperator::And => and, + VariadicOperator::Or => or, + }; + exprs + .iter() + .map(|expr| evaluate_expression(expr, batch, Some(&DataType::BOOLEAN))) + .reduce(|l, r| { + Ok(reducer(downcast_to_bool(&l?)?, downcast_to_bool(&r?)?) + .map(wrap_comparison_result)?) + }) + .transpose()? + .ok_or(Error::Generic("empty expression".to_string())) + } + (NullIf { expr, if_expr }, _) => { + let expr_arr = evaluate_expression(expr.as_ref(), batch, None)?; + let if_expr_arr = + evaluate_expression(if_expr.as_ref(), batch, Some(&DataType::BOOLEAN))?; + let if_expr_arr = downcast_to_bool(&if_expr_arr)?; + Ok(nullif(&expr_arr, if_expr_arr)?) + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::Int32Array; + use arrow_schema::{DataType, Field, Fields, Schema}; + use std::ops::{Add, Div, Mul, Sub}; + + #[test] + fn test_extract_column() { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = + RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(values.clone())]).unwrap(); + let column = Expression::Column("a".to_string()); + + let results = evaluate_expression(&column, &batch, None).unwrap(); + assert_eq!(results.as_ref(), &values); + + let schema = Schema::new(vec![Field::new( + "b", + DataType::Struct(Fields::from(vec![Field::new("a", DataType::Int32, false)])), + false, + )]); + + let struct_values: ArrayRef = Arc::new(values.clone()); + let struct_array = StructArray::from(vec![( + Arc::new(Field::new("a", DataType::Int32, false)), + struct_values, + )]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(struct_array.clone())], + ) + .unwrap(); + let column = Expression::Column("b.a".to_string()); + let results = evaluate_expression(&column, &batch, None).unwrap(); + assert_eq!(results.as_ref(), &values); + } + + #[test] + fn test_binary_op_scalar() { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(values)]).unwrap(); + let column = Expression::Column("a".to_string()); + + let expression = Box::new(column.clone().add(Expression::Literal(Scalar::Integer(1)))); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![2, 3, 4])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().sub(Expression::Literal(Scalar::Integer(1)))); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![0, 1, 2])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().mul(Expression::Literal(Scalar::Integer(2)))); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![2, 4, 6])); + assert_eq!(results.as_ref(), expected.as_ref()); + + // TODO handle type casting + let expression = Box::new(column.div(Expression::Literal(Scalar::Integer(1)))); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![1, 2, 3])); + assert_eq!(results.as_ref(), expected.as_ref()) + } + + #[test] + fn test_binary_op() { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(values.clone()), Arc::new(values)], + ) + .unwrap(); + let column_a = Expression::Column("a".to_string()); + let column_b = Expression::Column("b".to_string()); + + let expression = Box::new(column_a.clone().add(column_b.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![2, 4, 6])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column_a.clone().sub(column_b.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![0, 0, 0])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column_a.clone().mul(column_b)); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(Int32Array::from(vec![1, 4, 9])); + assert_eq!(results.as_ref(), expected.as_ref()); + } + + #[test] + fn test_binary_cmp() { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(values)]).unwrap(); + let column = Expression::Column("a".to_string()); + let lit = Expression::Literal(Scalar::Integer(2)); + + let expression = Box::new(column.clone().lt(lit.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().lt_eq(lit.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().gt(lit.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, false, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().gt_eq(lit.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, true, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().eq(lit.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().ne(lit.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + } + + #[test] + fn test_logical() { + let schema = Schema::new(vec![ + Field::new("a", DataType::Boolean, false), + Field::new("b", DataType::Boolean, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(BooleanArray::from(vec![true, false])), + Arc::new(BooleanArray::from(vec![false, true])), + ], + ) + .unwrap(); + let column_a = Expression::Column("a".to_string()); + let column_b = Expression::Column("b".to_string()); + + let expression = Box::new(column_a.clone().and(column_b.clone())); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new( + column_a + .clone() + .and(Expression::literal(Scalar::Boolean(true))), + ); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column_a.clone().or(column_b)); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new( + column_a + .clone() + .or(Expression::literal(Scalar::Boolean(false))), + ); + let results = evaluate_expression(&expression, &batch, None).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + } +} diff --git a/crates/core/src/kernel/expressions/mod.rs b/crates/core/src/kernel/expressions/mod.rs new file mode 100644 index 0000000000..b7912681ec --- /dev/null +++ b/crates/core/src/kernel/expressions/mod.rs @@ -0,0 +1,478 @@ +//! expressions. + +use std::collections::HashSet; +use std::fmt::{Display, Formatter}; +use std::sync::Arc; + +use arrow_array::{ArrayRef, RecordBatch}; +use arrow_schema::Schema as ArrowSchema; +use itertools::Itertools; + +use self::eval::evaluate_expression; +use super::{DataType, DeltaResult, SchemaRef}; + +pub use self::scalars::*; + +mod eval; +mod scalars; + +/// Interface for implementing an Expression evaluator. +/// +/// It contains one Expression which can be evaluated on multiple ColumnarBatches. +/// Connectors can implement this interface to optimize the evaluation using the +/// connector specific capabilities. +pub trait ExpressionEvaluator { + /// Evaluate the expression on given ColumnarBatch data. + /// + /// Contains one value for each row of the input. + /// The data type of the output is same as the type output of the expression this evaluator is using. + fn evaluate(&self, batch: &RecordBatch) -> DeltaResult; +} + +/// Provides expression evaluation capability to Delta Kernel. +/// +/// Delta Kernel can use this client to evaluate predicate on partition filters, +/// fill up partition column values and any computation on data using Expressions. +pub trait ExpressionHandler { + /// Create an [`ExpressionEvaluator`] that can evaluate the given [`Expression`] + /// on columnar batches with the given [`Schema`] to produce data of [`DataType`]. + /// + /// # Parameters + /// + /// - `schema`: Schema of the input data. + /// - `expression`: Expression to evaluate. + /// - `output_type`: Expected result data type. + /// + /// [`Schema`]: crate::schema::StructType + /// [`DataType`]: crate::schema::DataType + fn get_evaluator( + &self, + schema: SchemaRef, + expression: Expression, + output_type: DataType, + ) -> Arc; +} + +/// Default implementation of [`ExpressionHandler`] that uses [`evaluate_expression`] +#[derive(Debug)] +pub struct ArrowExpressionHandler {} + +impl ExpressionHandler for ArrowExpressionHandler { + fn get_evaluator( + &self, + schema: SchemaRef, + expression: Expression, + output_type: DataType, + ) -> Arc { + Arc::new(DefaultExpressionEvaluator { + input_schema: schema, + expression: Box::new(expression), + output_type, + }) + } +} + +/// Default implementation of [`ExpressionEvaluator`] that uses [`evaluate_expression`] +#[derive(Debug)] +pub struct DefaultExpressionEvaluator { + input_schema: SchemaRef, + expression: Box, + output_type: DataType, +} + +impl ExpressionEvaluator for DefaultExpressionEvaluator { + fn evaluate(&self, batch: &RecordBatch) -> DeltaResult { + let _input_schema: ArrowSchema = self.input_schema.as_ref().try_into()?; + // TODO: make sure we have matching schemas for validation + // if batch.schema().as_ref() != &input_schema { + // return Err(Error::Generic(format!( + // "input schema does not match batch schema: {:?} != {:?}", + // input_schema, + // batch.schema() + // ))); + // }; + evaluate_expression(&self.expression, batch, Some(&self.output_type)) + } +} + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +/// A binary operator. +pub enum BinaryOperator { + /// Arithmetic Plus + Plus, + /// Arithmetic Minus + Minus, + /// Arithmetic Multiply + Multiply, + /// Arithmetic Divide + Divide, + /// Comparison Less Than + LessThan, + /// Comparison Less Than Or Equal + LessThanOrEqual, + /// Comparison Greater Than + GreaterThan, + /// Comparison Greater Than Or Equal + GreaterThanOrEqual, + /// Comparison Equal + Equal, + /// Comparison Not Equal + NotEqual, +} + +/// Variadic operators +#[derive(Debug, Clone, PartialEq)] +pub enum VariadicOperator { + /// AND + And, + /// OR + Or, +} + +impl Display for BinaryOperator { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + // Self::And => write!(f, "AND"), + // Self::Or => write!(f, "OR"), + Self::Plus => write!(f, "+"), + Self::Minus => write!(f, "-"), + Self::Multiply => write!(f, "*"), + Self::Divide => write!(f, "/"), + Self::LessThan => write!(f, "<"), + Self::LessThanOrEqual => write!(f, "<="), + Self::GreaterThan => write!(f, ">"), + Self::GreaterThanOrEqual => write!(f, ">="), + Self::Equal => write!(f, "="), + Self::NotEqual => write!(f, "!="), + } + } +} + +#[derive(Debug, Clone, PartialEq)] +/// A unary operator. +pub enum UnaryOperator { + /// Unary Not + Not, + /// Unary Is Null + IsNull, +} + +/// A SQL expression. +/// +/// These expressions do not track or validate data types, other than the type +/// of literals. It is up to the expression evaluator to validate the +/// expression against a schema and add appropriate casts as required. +#[derive(Debug, Clone, PartialEq)] +pub enum Expression { + /// A literal value. + Literal(Scalar), + /// A column reference by name. + Column(String), + /// + Struct(Vec), + /// A binary operation. + BinaryOperation { + /// The operator. + op: BinaryOperator, + /// The left-hand side of the operation. + left: Box, + /// The right-hand side of the operation. + right: Box, + }, + /// A unary operation. + UnaryOperation { + /// The operator. + op: UnaryOperator, + /// The expression. + expr: Box, + }, + /// A variadic operation. + VariadicOperation { + /// The operator. + op: VariadicOperator, + /// The expressions. + exprs: Vec, + }, + /// A NULLIF expression. + NullIf { + /// The expression to evaluate. + expr: Box, + /// The expression to compare against. + if_expr: Box, + }, + // TODO: support more expressions, such as IS IN, LIKE, etc. +} + +impl> From for Expression { + fn from(value: T) -> Self { + Self::literal(value) + } +} + +impl Display for Expression { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::Literal(l) => write!(f, "{}", l), + Self::Column(name) => write!(f, "Column({})", name), + Self::Struct(exprs) => write!( + f, + "Struct({})", + &exprs.iter().map(|e| format!("{e}")).join(", ") + ), + Self::BinaryOperation { op, left, right } => write!(f, "{} {} {}", left, op, right), + Self::UnaryOperation { op, expr } => match op { + UnaryOperator::Not => write!(f, "NOT {}", expr), + UnaryOperator::IsNull => write!(f, "{} IS NULL", expr), + }, + Self::VariadicOperation { op, exprs } => match op { + VariadicOperator::And => { + write!( + f, + "AND({})", + &exprs.iter().map(|e| format!("{e}")).join(", ") + ) + } + VariadicOperator::Or => { + write!( + f, + "OR({})", + &exprs.iter().map(|e| format!("{e}")).join(", ") + ) + } + }, + Self::NullIf { expr, if_expr } => write!(f, "NULLIF({}, {})", expr, if_expr), + } + } +} + +impl Expression { + /// Returns a set of columns referenced by this expression. + pub fn references(&self) -> HashSet<&str> { + let mut set = HashSet::new(); + + for expr in self.walk() { + if let Self::Column(name) = expr { + set.insert(name.as_str()); + } + } + + set + } + + /// Create an new expression for a column reference + pub fn column(name: impl Into) -> Self { + Self::Column(name.into()) + } + + /// Create a new expression for a literal value + pub fn literal(value: impl Into) -> Self { + Self::Literal(value.into()) + } + + /// Create a new expression for a struct + pub fn struct_expr(exprs: impl IntoIterator) -> Self { + Self::Struct(exprs.into_iter().collect()) + } + + /// Create a new expression for a unary operation + pub fn unary(op: UnaryOperator, expr: impl Into) -> Self { + Self::UnaryOperation { + op, + expr: Box::new(expr.into()), + } + } + + /// Create a new expression for a binary operation + pub fn binary( + op: BinaryOperator, + lhs: impl Into, + rhs: impl Into, + ) -> Self { + Self::BinaryOperation { + op, + left: Box::new(lhs.into()), + right: Box::new(rhs.into()), + } + } + + /// Create a new expression for a variadic operation + pub fn variadic(op: VariadicOperator, other: impl IntoIterator) -> Self { + let mut exprs = other.into_iter().collect::>(); + if exprs.is_empty() { + // TODO this might break if we introduce new variadic operators? + return Self::literal(matches!(op, VariadicOperator::And)); + } + if exprs.len() == 1 { + return exprs.pop().unwrap(); + } + Self::VariadicOperation { op, exprs } + } + + /// Create a new expression `self == other` + pub fn eq(self, other: Self) -> Self { + Self::binary(BinaryOperator::Equal, self, other) + } + + /// Create a new expression `self != other` + pub fn ne(self, other: Self) -> Self { + Self::binary(BinaryOperator::NotEqual, self, other) + } + + /// Create a new expression `self < other` + pub fn lt(self, other: Self) -> Self { + Self::binary(BinaryOperator::LessThan, self, other) + } + + /// Create a new expression `self > other` + pub fn gt(self, other: Self) -> Self { + Self::binary(BinaryOperator::GreaterThan, self, other) + } + + /// Create a new expression `self >= other` + pub fn gt_eq(self, other: Self) -> Self { + Self::binary(BinaryOperator::GreaterThanOrEqual, self, other) + } + + /// Create a new expression `self <= other` + pub fn lt_eq(self, other: Self) -> Self { + Self::binary(BinaryOperator::LessThanOrEqual, self, other) + } + + /// Create a new expression `self AND other` + pub fn and(self, other: Self) -> Self { + self.and_many([other]) + } + + /// Create a new expression `self AND others` + pub fn and_many(self, other: impl IntoIterator) -> Self { + Self::variadic(VariadicOperator::And, std::iter::once(self).chain(other)) + } + + /// Create a new expression `self AND other` + pub fn or(self, other: Self) -> Self { + self.or_many([other]) + } + + /// Create a new expression `self OR other` + pub fn or_many(self, other: impl IntoIterator) -> Self { + Self::variadic(VariadicOperator::Or, std::iter::once(self).chain(other)) + } + + /// Create a new expression `self IS NULL` + pub fn is_null(self) -> Self { + Self::unary(UnaryOperator::IsNull, self) + } + + /// Create a new expression `NULLIF(self, other)` + pub fn null_if(self, other: Self) -> Self { + Self::NullIf { + expr: Box::new(self), + if_expr: Box::new(other), + } + } + + fn walk(&self) -> impl Iterator + '_ { + let mut stack = vec![self]; + std::iter::from_fn(move || { + let expr = stack.pop()?; + match expr { + Self::Literal(_) => {} + Self::Column { .. } => {} + Self::Struct(exprs) => { + stack.extend(exprs.iter()); + } + Self::BinaryOperation { left, right, .. } => { + stack.push(left); + stack.push(right); + } + Self::UnaryOperation { expr, .. } => { + stack.push(expr); + } + Self::VariadicOperation { op, exprs } => match op { + VariadicOperator::And | VariadicOperator::Or => { + stack.extend(exprs.iter()); + } + }, + Self::NullIf { expr, if_expr } => { + stack.push(expr); + stack.push(if_expr); + } + } + Some(expr) + }) + } +} + +impl std::ops::Add for Expression { + type Output = Self; + + fn add(self, rhs: Expression) -> Self::Output { + Self::binary(BinaryOperator::Plus, self, rhs) + } +} + +impl std::ops::Sub for Expression { + type Output = Self; + + fn sub(self, rhs: Expression) -> Self::Output { + Self::binary(BinaryOperator::Minus, self, rhs) + } +} + +impl std::ops::Mul for Expression { + type Output = Self; + + fn mul(self, rhs: Expression) -> Self::Output { + Self::binary(BinaryOperator::Multiply, self, rhs) + } +} + +impl std::ops::Div for Expression { + type Output = Self; + + fn div(self, rhs: Expression) -> Self::Output { + Self::binary(BinaryOperator::Divide, self, rhs) + } +} + +#[cfg(test)] +mod tests { + use super::Expression as Expr; + + #[test] + fn test_expression_format() { + let col_ref = Expr::column("x"); + let cases = [ + (col_ref.clone(), "Column(x)"), + (col_ref.clone().eq(Expr::literal(2)), "Column(x) = 2"), + ( + col_ref + .clone() + .gt_eq(Expr::literal(2)) + .and(col_ref.clone().lt_eq(Expr::literal(10))), + "AND(Column(x) >= 2, Column(x) <= 10)", + ), + ( + col_ref + .clone() + .gt(Expr::literal(2)) + .or(col_ref.clone().lt(Expr::literal(10))), + "OR(Column(x) > 2, Column(x) < 10)", + ), + ( + (col_ref.clone() - Expr::literal(4)).lt(Expr::literal(10)), + "Column(x) - 4 < 10", + ), + ( + (col_ref.clone() + Expr::literal(4)) / Expr::literal(10) * Expr::literal(42), + "Column(x) + 4 / 10 * 42", + ), + (col_ref.eq(Expr::literal("foo")), "Column(x) = 'foo'"), + ]; + + for (expr, expected) in cases { + let result = format!("{}", expr); + assert_eq!(result, expected); + } + } +} diff --git a/crates/core/src/kernel/expressions/scalars.rs b/crates/core/src/kernel/expressions/scalars.rs new file mode 100644 index 0000000000..d29cccb022 --- /dev/null +++ b/crates/core/src/kernel/expressions/scalars.rs @@ -0,0 +1,556 @@ +//! Scalar values for use in expressions. + +use std::cmp::Ordering; +use std::fmt::{Display, Formatter}; + +use arrow_array::Array; +use arrow_schema::TimeUnit; +use chrono::{DateTime, NaiveDate, NaiveDateTime, TimeZone, Utc}; +use object_store::path::Path; + +use crate::kernel::{DataType, Error, PrimitiveType, StructField}; +use crate::NULL_PARTITION_VALUE_DATA_PATH; + +/// A single value, which can be null. Used for representing literal values +/// in [Expressions][crate::expressions::Expression]. +#[derive(Debug, Clone, PartialEq)] +pub enum Scalar { + /// 32bit integer + Integer(i32), + /// 64bit integer + Long(i64), + /// 16bit integer + Short(i16), + /// 8bit integer + Byte(i8), + /// 32bit floating point + Float(f32), + /// 64bit floating point + Double(f64), + /// utf-8 encoded string. + String(String), + /// true or false value + Boolean(bool), + /// Microsecond precision timestamp, adjusted to UTC. + Timestamp(i64), + /// Microsecond precision timestamp, with no timezone. + TimestampNtz(i64), + /// Date stored as a signed 32bit int days since UNIX epoch 1970-01-01 + Date(i32), + /// Binary data + Binary(Vec), + /// Decimal value + Decimal(i128, u8, i8), + /// Null value with a given data type. + Null(DataType), + /// Struct value + Struct(Vec, Vec), +} + +impl Scalar { + /// Returns the data type of this scalar. + pub fn data_type(&self) -> DataType { + match self { + Self::Integer(_) => DataType::Primitive(PrimitiveType::Integer), + Self::Long(_) => DataType::Primitive(PrimitiveType::Long), + Self::Short(_) => DataType::Primitive(PrimitiveType::Short), + Self::Byte(_) => DataType::Primitive(PrimitiveType::Byte), + Self::Float(_) => DataType::Primitive(PrimitiveType::Float), + Self::Double(_) => DataType::Primitive(PrimitiveType::Double), + Self::String(_) => DataType::Primitive(PrimitiveType::String), + Self::Boolean(_) => DataType::Primitive(PrimitiveType::Boolean), + Self::Timestamp(_) => DataType::Primitive(PrimitiveType::Timestamp), + Self::TimestampNtz(_) => DataType::Primitive(PrimitiveType::TimestampNtz), + Self::Date(_) => DataType::Primitive(PrimitiveType::Date), + Self::Binary(_) => DataType::Primitive(PrimitiveType::Binary), + Self::Decimal(_, precision, scale) => DataType::decimal(*precision, *scale), + Self::Null(data_type) => data_type.clone(), + Self::Struct(_, fields) => DataType::struct_type(fields.clone()), + } + } + + /// Returns true if this scalar is null. + pub fn is_null(&self) -> bool { + matches!(self, Self::Null(_)) + } + + /// Serializes this scalar as a string. + pub fn serialize(&self) -> String { + match self { + Self::String(s) => s.to_owned(), + Self::Byte(b) => b.to_string(), + Self::Short(s) => s.to_string(), + Self::Integer(i) => i.to_string(), + Self::Long(l) => l.to_string(), + Self::Float(f) => f.to_string(), + Self::Double(d) => d.to_string(), + Self::Boolean(b) => { + if *b { + "true".to_string() + } else { + "false".to_string() + } + } + Self::TimestampNtz(ts) | Self::Timestamp(ts) => { + let ts = Utc.timestamp_micros(*ts).single().unwrap(); + ts.format("%Y-%m-%d %H:%M:%S%.6f").to_string() + } + Self::Date(days) => { + let date = Utc.from_utc_datetime( + &NaiveDateTime::from_timestamp_opt(*days as i64 * 24 * 3600, 0).unwrap(), + ); + date.format("%Y-%m-%d").to_string() + } + Self::Decimal(value, _, scale) => match scale.cmp(&0) { + Ordering::Equal => value.to_string(), + Ordering::Greater => { + let scalar_multiple = 10_i128.pow(*scale as u32); + let mut s = String::new(); + s.push_str((value / scalar_multiple).to_string().as_str()); + s.push('.'); + s.push_str(&format!( + "{:0>scale$}", + value % scalar_multiple, + scale = *scale as usize + )); + s + } + Ordering::Less => { + let mut s = value.to_string(); + for _ in 0..(scale.abs()) { + s.push('0'); + } + s + } + }, + Self::Binary(val) => create_escaped_binary_string(val.as_slice()), + Self::Null(_) => "null".to_string(), + Self::Struct(_, _) => todo!("serializing struct values is not yet supported"), + } + } + + /// Serializes this scalar as a string for use in hive partition file names. + pub fn serialize_encoded(&self) -> String { + if self.is_null() { + return NULL_PARTITION_VALUE_DATA_PATH.to_string(); + } + Path::from(self.serialize()).to_string() + } + + /// Create a [`Scalar`] form a row in an arrow array. + pub fn from_array(arr: &dyn Array, index: usize) -> Option { + use arrow_array::*; + use arrow_schema::DataType::*; + + if arr.len() <= index { + return None; + } + if arr.is_null(index) { + return Some(Self::Null(arr.data_type().try_into().ok()?)); + } + + match arr.data_type() { + Utf8 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::String(v.value(index).to_string())), + LargeUtf8 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::String(v.value(index).to_string())), + Boolean => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Boolean(v.value(index))), + Binary => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Binary(v.value(index).to_vec())), + LargeBinary => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Binary(v.value(index).to_vec())), + FixedSizeBinary(_) => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Binary(v.value(index).to_vec())), + Int8 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Byte(v.value(index))), + Int16 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Short(v.value(index))), + Int32 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Integer(v.value(index))), + Int64 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Long(v.value(index))), + UInt8 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Byte(v.value(index) as i8)), + UInt16 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Short(v.value(index) as i16)), + UInt32 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Integer(v.value(index) as i32)), + UInt64 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Long(v.value(index) as i64)), + Float32 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Float(v.value(index))), + Float64 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Double(v.value(index))), + Decimal128(precision, scale) => { + arr.as_any().downcast_ref::().map(|v| { + let value = v.value(index); + Self::Decimal(value, *precision, *scale) + }) + } + Date32 => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Date(v.value(index))), + // TODO handle timezones when implementing timestamp ntz feature. + Timestamp(TimeUnit::Microsecond, tz) => match tz { + None => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Timestamp(v.value(index))), + Some(tz_str) if tz_str.as_ref() == "UTC" => arr + .as_any() + .downcast_ref::() + .map(|v| Self::Timestamp(v.clone().with_timezone("UTC").value(index))), + _ => None, + }, + Struct(fields) => { + let struct_fields = fields + .iter() + .flat_map(|f| TryFrom::try_from(f.as_ref())) + .collect::>(); + let values = arr + .as_any() + .downcast_ref::() + .and_then(|struct_arr| { + struct_fields + .iter() + .map(|f: &StructField| { + struct_arr + .column_by_name(f.name()) + .and_then(|c| Self::from_array(c.as_ref(), index)) + }) + .collect::>>() + })?; + if struct_fields.len() != values.len() { + return None; + } + Some(Self::Struct(values, struct_fields)) + } + Float16 + | Decimal256(_, _) + | List(_) + | LargeList(_) + | FixedSizeList(_, _) + | Map(_, _) + | Date64 + | Timestamp(_, _) + | Time32(_) + | Time64(_) + | Duration(_) + | Interval(_) + | Dictionary(_, _) + | RunEndEncoded(_, _) + | Union(_, _) + | Null => None, + } + } +} + +impl PartialOrd for Scalar { + fn partial_cmp(&self, other: &Self) -> Option { + use Scalar::*; + match (self, other) { + (Null(_), Null(_)) => Some(Ordering::Equal), + (Integer(a), Integer(b)) => a.partial_cmp(b), + (Long(a), Long(b)) => a.partial_cmp(b), + (Short(a), Short(b)) => a.partial_cmp(b), + (Byte(a), Byte(b)) => a.partial_cmp(b), + (Float(a), Float(b)) => a.partial_cmp(b), + (Double(a), Double(b)) => a.partial_cmp(b), + (String(a), String(b)) => a.partial_cmp(b), + (Boolean(a), Boolean(b)) => a.partial_cmp(b), + (Timestamp(a), Timestamp(b)) => a.partial_cmp(b), + (TimestampNtz(a), TimestampNtz(b)) => a.partial_cmp(b), + (Date(a), Date(b)) => a.partial_cmp(b), + (Binary(a), Binary(b)) => a.partial_cmp(b), + (Decimal(a, _, _), Decimal(b, _, _)) => a.partial_cmp(b), + (Struct(a, _), Struct(b, _)) => a.partial_cmp(b), + // TODO should we make an assumption about the ordering of nulls? + // rigth now this is only used for internal purposes. + (Null(_), _) => Some(Ordering::Less), + (_, Null(_)) => Some(Ordering::Greater), + _ => None, + } + } +} + +impl Display for Scalar { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::Integer(i) => write!(f, "{}", i), + Self::Long(i) => write!(f, "{}", i), + Self::Short(i) => write!(f, "{}", i), + Self::Byte(i) => write!(f, "{}", i), + Self::Float(fl) => write!(f, "{}", fl), + Self::Double(fl) => write!(f, "{}", fl), + Self::String(s) => write!(f, "'{}'", s), + Self::Boolean(b) => write!(f, "{}", b), + Self::Timestamp(ts) => write!(f, "{}", ts), + Self::TimestampNtz(ts) => write!(f, "{}", ts), + Self::Date(d) => write!(f, "{}", d), + Self::Binary(b) => write!(f, "{:?}", b), + Self::Decimal(value, _, scale) => match scale.cmp(&0) { + Ordering::Equal => { + write!(f, "{}", value) + } + Ordering::Greater => { + let scalar_multiple = 10_i128.pow(*scale as u32); + write!(f, "{}", value / scalar_multiple)?; + write!(f, ".")?; + write!( + f, + "{:0>scale$}", + value % scalar_multiple, + scale = *scale as usize + ) + } + Ordering::Less => { + write!(f, "{}", value)?; + for _ in 0..(scale.abs()) { + write!(f, "0")?; + } + Ok(()) + } + }, + Self::Null(_) => write!(f, "null"), + Self::Struct(values, fields) => { + write!(f, "{{")?; + for (i, (value, field)) in values.iter().zip(fields.iter()).enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{}: {}", field.name, value)?; + } + write!(f, "}}") + } + } + } +} + +impl From for Scalar { + fn from(i: i32) -> Self { + Self::Integer(i) + } +} + +impl From for Scalar { + fn from(i: i64) -> Self { + Self::Long(i) + } +} + +impl From for Scalar { + fn from(b: bool) -> Self { + Self::Boolean(b) + } +} + +impl From<&str> for Scalar { + fn from(s: &str) -> Self { + Self::String(s.into()) + } +} + +impl From for Scalar { + fn from(value: String) -> Self { + Self::String(value) + } +} + +// TODO: add more From impls + +impl PrimitiveType { + fn data_type(&self) -> DataType { + DataType::Primitive(self.clone()) + } + + /// Parses a string into a scalar value. + pub fn parse_scalar(&self, raw: &str) -> Result { + use PrimitiveType::*; + + lazy_static::lazy_static! { + static ref UNIX_EPOCH: DateTime = DateTime::from_timestamp(0, 0).unwrap(); + } + + if raw.is_empty() || raw == NULL_PARTITION_VALUE_DATA_PATH { + return Ok(Scalar::Null(self.data_type())); + } + + match self { + String => Ok(Scalar::String(raw.to_string())), + Byte => self.str_parse_scalar(raw, Scalar::Byte), + Short => self.str_parse_scalar(raw, Scalar::Short), + Integer => self.str_parse_scalar(raw, Scalar::Integer), + Long => self.str_parse_scalar(raw, Scalar::Long), + Float => self.str_parse_scalar(raw, Scalar::Float), + Double => self.str_parse_scalar(raw, Scalar::Double), + Boolean => { + if raw.eq_ignore_ascii_case("true") { + Ok(Scalar::Boolean(true)) + } else if raw.eq_ignore_ascii_case("false") { + Ok(Scalar::Boolean(false)) + } else { + Err(self.parse_error(raw)) + } + } + Date => { + let date = NaiveDate::parse_from_str(raw, "%Y-%m-%d") + .map_err(|_| self.parse_error(raw))? + .and_hms_opt(0, 0, 0) + .ok_or(self.parse_error(raw))?; + let date = Utc.from_utc_datetime(&date); + let days = date.signed_duration_since(*UNIX_EPOCH).num_days() as i32; + Ok(Scalar::Date(days)) + } + Timestamp => { + let timestamp = NaiveDateTime::parse_from_str(raw, "%Y-%m-%d %H:%M:%S%.f") + .map_err(|_| self.parse_error(raw))?; + let timestamp = Utc.from_utc_datetime(×tamp); + let micros = timestamp + .signed_duration_since(*UNIX_EPOCH) + .num_microseconds() + .ok_or(self.parse_error(raw))?; + Ok(Scalar::Timestamp(micros)) + } + TimestampNtz => { + let timestamp = NaiveDateTime::parse_from_str(raw, "%Y-%m-%d %H:%M:%S%.f") + .map_err(|_| self.parse_error(raw))?; + let timestamp = Utc.from_utc_datetime(×tamp); + let micros = timestamp + .signed_duration_since(*UNIX_EPOCH) + .num_microseconds() + .ok_or(self.parse_error(raw))?; + Ok(Scalar::TimestampNtz(micros)) + } + Binary => { + let bytes = parse_escaped_binary_string(raw).map_err(|_| self.parse_error(raw))?; + Ok(Scalar::Binary(bytes)) + } + _ => todo!("parsing {:?} is not yet supported", self), + } + } + + fn parse_error(&self, raw: &str) -> Error { + Error::Parse(raw.to_string(), self.data_type()) + } + + fn str_parse_scalar( + &self, + raw: &str, + f: impl FnOnce(T) -> Scalar, + ) -> Result { + match raw.parse() { + Ok(val) => Ok(f(val)), + Err(..) => Err(self.parse_error(raw)), + } + } +} + +fn create_escaped_binary_string(data: &[u8]) -> String { + let mut escaped_string = String::new(); + for &byte in data { + // Convert each byte to its two-digit hexadecimal representation + let hex_representation = format!("{:04X}", byte); + // Append the hexadecimal representation with an escape sequence + escaped_string.push_str("\\u"); + escaped_string.push_str(&hex_representation); + } + escaped_string +} + +fn parse_escaped_binary_string(escaped_string: &str) -> Result, &'static str> { + let mut parsed_bytes = Vec::new(); + let mut chars = escaped_string.chars(); + + while let Some(ch) = chars.next() { + if ch == '\\' { + // Check for the escape sequence "\\u" indicating a hexadecimal value + if chars.next() == Some('u') { + // Read two hexadecimal digits and convert to u8 + if let (Some(digit1), Some(digit2), Some(digit3), Some(digit4)) = + (chars.next(), chars.next(), chars.next(), chars.next()) + { + if let Ok(byte) = + u8::from_str_radix(&format!("{}{}{}{}", digit1, digit2, digit3, digit4), 16) + { + parsed_bytes.push(byte); + } else { + return Err("Error parsing hexadecimal value"); + } + } else { + return Err("Incomplete escape sequence"); + } + } else { + // Unrecognized escape sequence + return Err("Unrecognized escape sequence"); + } + } else { + // Regular character, convert to u8 and push into the result vector + parsed_bytes.push(ch as u8); + } + } + + Ok(parsed_bytes) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_binary_roundtrip() { + let scalar = Scalar::Binary(vec![0, 1, 2, 3, 4, 5]); + let parsed = PrimitiveType::Binary + .parse_scalar(&scalar.serialize()) + .unwrap(); + assert_eq!(scalar, parsed); + } + + #[test] + fn test_decimal_display() { + let s = Scalar::Decimal(123456789, 9, 2); + assert_eq!(s.to_string(), "1234567.89"); + + let s = Scalar::Decimal(123456789, 9, 0); + assert_eq!(s.to_string(), "123456789"); + + let s = Scalar::Decimal(123456789, 9, 9); + assert_eq!(s.to_string(), "0.123456789"); + + let s = Scalar::Decimal(123, 9, -3); + assert_eq!(s.to_string(), "123000"); + } +} diff --git a/crates/core/src/kernel/mod.rs b/crates/core/src/kernel/mod.rs new file mode 100644 index 0000000000..876a09a33c --- /dev/null +++ b/crates/core/src/kernel/mod.rs @@ -0,0 +1,22 @@ +//! Delta Kernel module +//! +//! The Kernel module contains all the logic for reading and processing the Delta Lake transaction log. + +pub mod arrow; +pub mod error; +pub mod expressions; +pub mod models; +mod snapshot; + +pub use error::*; +pub use expressions::*; +pub use models::*; +pub use snapshot::*; + +/// 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/actions/types.rs b/crates/core/src/kernel/models/actions.rs similarity index 91% rename from crates/deltalake-core/src/kernel/actions/types.rs rename to crates/core/src/kernel/models/actions.rs index f38cbd51b2..f389102e52 100644 --- a/crates/deltalake-core/src/kernel/actions/types.rs +++ b/crates/core/src/kernel/models/actions.rs @@ -5,13 +5,12 @@ use std::str::FromStr; // use std::sync::Arc; // use roaring::RoaringTreemap; -use log::warn; use serde::{Deserialize, Serialize}; +use tracing::warn; use url::Url; -use super::super::schema::StructType; -use super::super::{error::Error, DeltaResult}; -use super::serde_path; +use super::schema::StructType; +use crate::kernel::{error::Error, DeltaResult}; #[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] /// Defines a file format used in table @@ -75,23 +74,21 @@ pub struct Metadata { impl Metadata { /// Create a new metadata action - pub fn new( - id: impl Into, - format: Format, - schema_string: impl Into, + pub fn try_new( + schema: StructType, partition_columns: impl IntoIterator>, - configuration: Option>>, - ) -> Self { - Self { - id: id.into(), - format, - schema_string: schema_string.into(), + configuration: HashMap>, + ) -> DeltaResult { + Ok(Self { + id: uuid::Uuid::new_v4().to_string(), + format: Default::default(), + schema_string: serde_json::to_string(&schema)?, partition_columns: partition_columns.into_iter().map(|c| c.into()).collect(), - configuration: configuration.unwrap_or_default(), + configuration, name: None, description: None, created_time: None, - } + }) } /// set the table name in the metadata action @@ -130,9 +127,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>, } @@ -176,7 +175,7 @@ pub enum ReaderFeatures { /// Deletion vectors for merge, update, delete DeletionVectors, /// timestamps without timezone support - #[serde(alias = "timestampNtz")] + #[serde(rename = "timestampNtz")] TimestampWithoutTimezone, /// version 2 of checkpointing V2Checkpoint, @@ -185,13 +184,14 @@ pub enum ReaderFeatures { Other(String), } -#[cfg(all(not(feature = "parquet2"), feature = "parquet"))] impl From<&parquet::record::Field> for ReaderFeatures { fn from(value: &parquet::record::Field) -> Self { match value { parquet::record::Field::Str(feature) => match feature.as_str() { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeletionVectors, + "deletionVectors" | "delta.enableDeletionVectors" => { + ReaderFeatures::DeletionVectors + } "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -261,7 +261,7 @@ pub enum WriterFeatures { /// Row tracking on tables RowTracking, /// timestamps without timezone support - #[serde(alias = "timestampNtz")] + #[serde(rename = "timestampNtz")] TimestampWithoutTimezone, /// domain specific metadata DomainMetadata, @@ -283,15 +283,15 @@ impl From for WriterFeatures { impl From<&str> for WriterFeatures { fn from(value: &str) -> Self { match value { - "appendOnly" => WriterFeatures::AppendOnly, - "invariants" => WriterFeatures::Invariants, - "checkConstraints" => WriterFeatures::CheckConstraints, - "changeDataFeed" => WriterFeatures::ChangeDataFeed, + "appendOnly" | "delta.appendOnly" => WriterFeatures::AppendOnly, + "invariants" | "delta.invariants" => WriterFeatures::Invariants, + "checkConstraints" | "delta.checkConstraints" => WriterFeatures::CheckConstraints, + "changeDataFeed" | "delta.enableChangeDataFeed" => WriterFeatures::ChangeDataFeed, "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeletionVectors, - "rowTracking" => WriterFeatures::RowTracking, + "deletionVectors" | "delta.enableDeletionVectors" => WriterFeatures::DeletionVectors, + "rowTracking" | "delta.enableRowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, "v2Checkpoint" => WriterFeatures::V2Checkpoint, @@ -328,7 +328,6 @@ impl fmt::Display for WriterFeatures { } } -#[cfg(all(not(feature = "parquet2"), feature = "parquet"))] impl From<&parquet::record::Field> for WriterFeatures { fn from(value: &parquet::record::Field) -> Self { match value { @@ -440,19 +439,6 @@ pub struct DeletionVectorDescriptor { } impl DeletionVectorDescriptor { - /// get a unique idenitfier for the deletion vector - pub fn unique_id(&self) -> String { - if let Some(offset) = self.offset { - format!( - "{}{}@{offset}", - self.storage_type.as_ref(), - self.path_or_inline_dv - ) - } else { - format!("{}{}", self.storage_type.as_ref(), self.path_or_inline_dv) - } - } - /// get the absolute path of the deletion vector pub fn absolute_path(&self, parent: &Url) -> DeltaResult> { match &self.storage_type { @@ -586,47 +572,13 @@ pub struct Add { /// The name of the clustering implementation pub clustering_provider: Option, - // TODO remove migration filds added to not do too many business logic changes in one PR - /// Partition values stored in raw parquet struct format. In this struct, the column names - /// correspond to the partition columns and the values are stored in their corresponding data - /// type. This is a required field when the table is partitioned and the table property - /// delta.checkpoint.writeStatsAsStruct is set to true. If the table is not partitioned, this - /// column can be omitted. - /// - /// This field is only available in add action records read from checkpoints - #[cfg(feature = "parquet")] - #[serde(skip_serializing, skip_deserializing)] - pub partition_values_parsed: Option, - /// Partition values parsed for parquet2 - #[cfg(feature = "parquet2")] - #[serde(skip_serializing, skip_deserializing)] - pub partition_values_parsed: Option, - /// Contains statistics (e.g., count, min/max values for columns) about the data in this file in /// raw parquet format. This field needs to be written when statistics are available and the /// table property: delta.checkpoint.writeStatsAsStruct is set to true. /// /// This field is only available in add action records read from checkpoints - #[cfg(feature = "parquet")] #[serde(skip_serializing, skip_deserializing)] pub stats_parsed: Option, - /// Stats parsed for parquet2 - #[cfg(feature = "parquet2")] - #[serde(skip_serializing, skip_deserializing)] - pub stats_parsed: Option, -} - -impl Add { - /// get the unique id of the deletion vector, if any - pub fn dv_unique_id(&self) -> Option { - self.deletion_vector.clone().map(|dv| dv.unique_id()) - } - - /// set the base row id of the add action - pub fn with_base_row_id(mut self, base_row_id: i64) -> Self { - self.base_row_id = Some(base_row_id); - self - } } /// Represents a tombstone (deleted file) in the Delta log. @@ -638,6 +590,7 @@ pub struct Remove { /// [RFC 2396 URI Generic Syntax], which needs to be decoded to get the data file path. /// /// [RFC 2396 URI Generic Syntax]: https://www.ietf.org/rfc/rfc2396.txt + #[serde(with = "serde_path")] pub path: String, /// When `false` the logical file must already be present in the table or the records @@ -679,13 +632,6 @@ pub struct Remove { pub default_row_commit_version: Option, } -impl Remove { - /// get the unique id of the deletion vector, if any - pub fn dv_unique_id(&self) -> Option { - self.deletion_vector.clone().map(|dv| dv.unique_id()) - } -} - /// Delta AddCDCFile action that describes a parquet CDC data file. #[derive(Serialize, Deserialize, Clone, Debug, Default, PartialEq, Eq)] #[serde(rename_all = "camelCase")] @@ -876,6 +822,68 @@ impl FromStr for IsolationLevel { } } +pub(crate) mod serde_path { + use std::str::Utf8Error; + + use percent_encoding::{percent_decode_str, percent_encode, AsciiSet, CONTROLS}; + use serde::{self, Deserialize, Deserializer, Serialize, Serializer}; + + pub fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + decode_path(&s).map_err(serde::de::Error::custom) + } + + pub fn serialize(value: &str, serializer: S) -> Result + where + S: Serializer, + { + let encoded = encode_path(value); + String::serialize(&encoded, serializer) + } + + pub const _DELIMITER: &str = "/"; + /// The path delimiter as a single byte + pub const _DELIMITER_BYTE: u8 = _DELIMITER.as_bytes()[0]; + + /// Characters we want to encode. + const INVALID: &AsciiSet = &CONTROLS + // The delimiter we are reserving for internal hierarchy + // .add(DELIMITER_BYTE) + // Characters AWS recommends avoiding for object keys + // https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingMetadata.html + .add(b'\\') + .add(b'{') + .add(b'^') + .add(b'}') + .add(b'%') + .add(b'`') + .add(b']') + .add(b'"') + .add(b'>') + .add(b'[') + // .add(b'~') + .add(b'<') + .add(b'#') + .add(b'|') + // Characters Google Cloud Storage recommends avoiding for object names + // https://cloud.google.com/storage/docs/naming-objects + .add(b'\r') + .add(b'\n') + .add(b'*') + .add(b'?'); + + fn encode_path(path: &str) -> String { + percent_encode(path.as_bytes(), INVALID).to_string() + } + + pub fn decode_path(path: &str) -> Result { + Ok(percent_decode_str(path).decode_utf8()?.to_string()) + } +} + #[cfg(test)] mod tests { use std::path::PathBuf; @@ -949,8 +957,8 @@ mod tests { let inline = dv_inline(); assert_eq!(None, inline.absolute_path(&parent).unwrap()); - let path = - std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + let path = std::fs::canonicalize(PathBuf::from("../test/tests/data/table-with-dv-small/")) + .unwrap(); let parent = url::Url::from_directory_path(path).unwrap(); let dv_url = parent .join("deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin") @@ -969,7 +977,7 @@ mod tests { // fn test_deletion_vector_read() { // let store = Arc::new(LocalFileSystem::new()); // let path = - // std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + // std::fs::canonicalize(PathBuf::from("../deltalake-test/tests/data/table-with-dv-small/")).unwrap(); // let parent = url::Url::from_directory_path(path).unwrap(); // let root = object_store::path::Path::from(parent.path()); // let fs_client = Arc::new(ObjectStoreFileSystemClient::new( diff --git a/crates/core/src/kernel/models/fields.rs b/crates/core/src/kernel/models/fields.rs new file mode 100644 index 0000000000..fa672aaefc --- /dev/null +++ b/crates/core/src/kernel/models/fields.rs @@ -0,0 +1,273 @@ +//! Schema definitions for action types + +use lazy_static::lazy_static; + +use super::schema::{ArrayType, DataType, MapType, StructField, StructType}; +use super::ActionType; + +impl ActionType { + /// Returns the type of the corresponding field in the delta log schema + pub(crate) fn schema_field(&self) -> &StructField { + match self { + Self::Metadata => &METADATA_FIELD, + Self::Protocol => &PROTOCOL_FIELD, + Self::CommitInfo => &COMMIT_INFO_FIELD, + Self::Add => &ADD_FIELD, + Self::Remove => &REMOVE_FIELD, + Self::Cdc => &CDC_FIELD, + Self::Txn => &TXN_FIELD, + Self::DomainMetadata => &DOMAIN_METADATA_FIELD, + Self::CheckpointMetadata => &CHECKPOINT_METADATA_FIELD, + Self::Sidecar => &SIDECAR_FIELD, + } + } +} + +lazy_static! { + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#change-metadata + static ref METADATA_FIELD: StructField = StructField::new( + "metaData", + StructType::new(vec![ + StructField::new("id", DataType::STRING, true), + StructField::new("name", DataType::STRING, true), + StructField::new("description", DataType::STRING, true), + StructField::new( + "format", + StructType::new(vec![ + StructField::new("provider", DataType::STRING, true), + StructField::new( + "options", + MapType::new( + DataType::STRING, + DataType::STRING, + true, + ), + false, + ), + ]), + false, + ), + StructField::new("schemaString", DataType::STRING, true), + StructField::new( + "partitionColumns", + ArrayType::new(DataType::STRING, false), + true, + ), + StructField::new("createdTime", DataType::LONG, true), + StructField::new( + "configuration", + MapType::new( + DataType::STRING, + DataType::STRING, + true, + ), + false, + ), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#protocol-evolution + static ref PROTOCOL_FIELD: StructField = StructField::new( + "protocol", + StructType::new(vec![ + StructField::new("minReaderVersion", DataType::INTEGER, true), + StructField::new("minWriterVersion", DataType::INTEGER, true), + StructField::new( + "readerFeatures", + ArrayType::new(DataType::STRING, true), + true, + ), + StructField::new( + "writerFeatures", + ArrayType::new(DataType::STRING, true), + true, + ), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#commit-provenance-information + static ref COMMIT_INFO_FIELD: StructField = StructField::new( + "commitInfo", + StructType::new(vec![ + StructField::new("timestamp", DataType::LONG, false), + StructField::new("operation", DataType::STRING, false), + StructField::new("isolationLevel", DataType::STRING, true), + StructField::new("isBlindAppend", DataType::BOOLEAN, true), + StructField::new("txnId", DataType::STRING, true), + StructField::new("readVersion", DataType::LONG, true), + StructField::new( + "operationParameters", + MapType::new( + DataType::STRING, + DataType::STRING, + true, + ), + true, + ), + StructField::new( + "operationMetrics", + MapType::new( + DataType::STRING, + DataType::STRING, + true, + ), + true, + ), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file + static ref ADD_FIELD: StructField = StructField::new( + "add", + StructType::new(vec![ + StructField::new("path", DataType::STRING, true), + partition_values_field(), + StructField::new("size", DataType::LONG, true), + StructField::new("modificationTime", DataType::LONG, true), + StructField::new("dataChange", DataType::BOOLEAN, true), + StructField::new("stats", DataType::STRING, true), + tags_field(), + deletion_vector_field(), + StructField::new("baseRowId", DataType::LONG, true), + StructField::new("defaultRowCommitVersion", DataType::LONG, true), + StructField::new("clusteringProvider", DataType::STRING, true), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file + static ref REMOVE_FIELD: StructField = StructField::new( + "remove", + StructType::new(vec![ + StructField::new("path", DataType::STRING, true), + StructField::new("deletionTimestamp", DataType::LONG, true), + StructField::new("dataChange", DataType::BOOLEAN, true), + StructField::new("extendedFileMetadata", DataType::BOOLEAN, true), + partition_values_field(), + StructField::new("size", DataType::LONG, true), + StructField::new("stats", DataType::STRING, true), + tags_field(), + deletion_vector_field(), + StructField::new("baseRowId", DataType::LONG, true), + StructField::new("defaultRowCommitVersion", DataType::LONG, true), + ]), + true, + ); + static ref REMOVE_FIELD_CHECKPOINT: StructField = StructField::new( + "remove", + StructType::new(vec![ + StructField::new("path", DataType::STRING, false), + StructField::new("deletionTimestamp", DataType::LONG, true), + StructField::new("dataChange", DataType::BOOLEAN, false), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-cdc-file + static ref CDC_FIELD: StructField = StructField::new( + "cdc", + StructType::new(vec![ + StructField::new("path", DataType::STRING, false), + partition_values_field(), + StructField::new("size", DataType::LONG, false), + StructField::new("dataChange", DataType::BOOLEAN, false), + tags_field(), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#transaction-identifiers + static ref TXN_FIELD: StructField = StructField::new( + "txn", + StructType::new(vec![ + StructField::new("appId", DataType::STRING, false), + StructField::new("version", DataType::LONG, false), + StructField::new("lastUpdated", DataType::LONG, true), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#domain-metadata + static ref DOMAIN_METADATA_FIELD: StructField = StructField::new( + "domainMetadata", + StructType::new(vec![ + StructField::new("domain", DataType::STRING, false), + StructField::new( + "configuration", + MapType::new( + DataType::STRING, + DataType::STRING, + true, + ), + true, + ), + StructField::new("removed", DataType::BOOLEAN, false), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#checkpoint-metadata + static ref CHECKPOINT_METADATA_FIELD: StructField = StructField::new( + "checkpointMetadata", + StructType::new(vec![ + StructField::new("flavor", DataType::STRING, false), + tags_field(), + ]), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#sidecar-file-information + static ref SIDECAR_FIELD: StructField = StructField::new( + "sidecar", + StructType::new(vec![ + StructField::new("path", DataType::STRING, false), + StructField::new("sizeInBytes", DataType::LONG, true), + StructField::new("modificationTime", DataType::LONG, false), + StructField::new("type", DataType::STRING, false), + tags_field(), + ]), + true, + ); + + static ref LOG_SCHEMA: StructType = StructType::new( + vec![ + ADD_FIELD.clone(), + CDC_FIELD.clone(), + COMMIT_INFO_FIELD.clone(), + DOMAIN_METADATA_FIELD.clone(), + METADATA_FIELD.clone(), + PROTOCOL_FIELD.clone(), + REMOVE_FIELD.clone(), + TXN_FIELD.clone(), + ] + ); +} + +fn tags_field() -> StructField { + StructField::new( + "tags", + MapType::new(DataType::STRING, DataType::STRING, true), + true, + ) +} + +fn partition_values_field() -> StructField { + StructField::new( + "partitionValues", + MapType::new(DataType::STRING, DataType::STRING, true), + true, + ) +} + +fn deletion_vector_field() -> StructField { + StructField::new( + "deletionVector", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("storageType", DataType::STRING, true), + StructField::new("pathOrInlineDv", DataType::STRING, true), + StructField::new("offset", DataType::INTEGER, true), + StructField::new("sizeInBytes", DataType::INTEGER, true), + StructField::new("cardinality", DataType::LONG, true), + ]))), + true, + ) +} + +#[cfg(test)] +pub(crate) fn log_schema() -> &'static StructType { + &LOG_SCHEMA +} diff --git a/crates/deltalake-core/src/kernel/actions/mod.rs b/crates/core/src/kernel/models/mod.rs similarity index 55% rename from crates/deltalake-core/src/kernel/actions/mod.rs rename to crates/core/src/kernel/models/mod.rs index 637d520c41..eda7e6fb60 100644 --- a/crates/deltalake-core/src/kernel/actions/mod.rs +++ b/crates/core/src/kernel/models/mod.rs @@ -7,11 +7,12 @@ use std::collections::HashMap; use serde::{Deserialize, Serialize}; -pub(crate) mod schemas; -mod serde_path; -pub(crate) mod types; +pub(crate) mod actions; +pub(crate) mod fields; +mod schema; -pub use types::*; +pub use actions::*; +pub use schema::*; #[derive(Debug)] /// The type of action that was performed on the table @@ -62,3 +63,67 @@ impl Action { }) } } + +impl From for Action { + fn from(a: Add) -> Self { + Self::Add(a) + } +} + +impl From for Action { + fn from(a: Remove) -> Self { + Self::Remove(a) + } +} + +impl From for Action { + fn from(a: AddCDCFile) -> Self { + Self::Cdc(a) + } +} + +impl From for Action { + fn from(a: Metadata) -> Self { + Self::Metadata(a) + } +} + +impl From for Action { + fn from(a: Protocol) -> Self { + Self::Protocol(a) + } +} + +impl From for Action { + fn from(a: Txn) -> Self { + Self::Txn(a) + } +} + +impl From for Action { + fn from(a: CommitInfo) -> Self { + Self::CommitInfo(a) + } +} + +impl From for Action { + fn from(a: DomainMetadata) -> Self { + Self::DomainMetadata(a) + } +} + +impl Action { + /// Get the action type + pub fn action_type(&self) -> ActionType { + match self { + Self::Add(_) => ActionType::Add, + Self::Remove(_) => ActionType::Remove, + Self::Cdc(_) => ActionType::Cdc, + Self::Metadata(_) => ActionType::Metadata, + Self::Protocol(_) => ActionType::Protocol, + Self::Txn(_) => ActionType::Txn, + Self::CommitInfo(_) => ActionType::CommitInfo, + Self::DomainMetadata(_) => ActionType::DomainMetadata, + } + } +} diff --git a/crates/deltalake-core/src/kernel/schema.rs b/crates/core/src/kernel/models/schema.rs similarity index 78% rename from crates/deltalake-core/src/kernel/schema.rs rename to crates/core/src/kernel/models/schema.rs index bc83c05070..84e5967f12 100644 --- a/crates/deltalake-core/src/kernel/schema.rs +++ b/crates/core/src/kernel/models/schema.rs @@ -9,7 +9,8 @@ use std::{collections::HashMap, fmt::Display}; use serde::{Deserialize, Serialize}; use serde_json::Value; -use super::error::Error; +use crate::kernel::error::Error; +use crate::kernel::DataCheck; /// Type alias for a top level schema pub type Schema = StructType; @@ -24,6 +25,8 @@ pub enum MetadataValue { Number(i32), /// A string value String(String), + /// A Boolean value + Boolean(bool), } impl From for MetadataValue { @@ -44,6 +47,12 @@ impl From for MetadataValue { } } +impl From for MetadataValue { + fn from(value: bool) -> Self { + Self::Boolean(value) + } +} + impl From for MetadataValue { fn from(value: Value) -> Self { Self::String(value.to_string()) @@ -97,6 +106,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)] @@ -115,6 +134,8 @@ pub struct StructField { impl Hash for StructField { fn hash(&self, state: &mut H) { self.name.hash(state); + self.data_type.hash(state); + self.nullable.hash(state); } } @@ -128,10 +149,10 @@ impl Eq for StructField {} impl StructField { /// Creates a new field - pub fn new(name: impl Into, data_type: DataType, nullable: bool) -> Self { + pub fn new(name: impl Into, data_type: impl Into, nullable: bool) -> Self { Self { name: name.into(), - data_type, + data_type: data_type.into(), nullable, metadata: HashMap::default(), } @@ -173,6 +194,7 @@ impl StructField { let phys_name = self.get_config_value(&ColumnMetadataKey::ColumnMappingPhysicalName); match phys_name { None => Ok(&self.name), + Some(MetadataValue::Boolean(_)) => Ok(&self.name), Some(MetadataValue::String(s)) => Ok(s), Some(MetadataValue::Number(_)) => Err(Error::MetadataError( "Unexpected type for physical name".to_string(), @@ -195,7 +217,7 @@ impl StructField { /// A struct is used to represent both the top-level schema of the table /// as well as struct columns that contain nested columns. -#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] +#[derive(Debug, Serialize, Deserialize, PartialEq, Clone, Eq, Hash)] pub struct StructType { #[serde(rename = "type")] /// The type of this struct @@ -314,7 +336,52 @@ impl StructType { } } -#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] +impl FromIterator for StructType { + fn from_iter>(iter: T) -> Self { + Self { + type_name: "struct".into(), + fields: iter.into_iter().collect(), + } + } +} + +impl<'a> FromIterator<&'a StructField> for StructType { + fn from_iter>(iter: T) -> Self { + Self { + type_name: "struct".into(), + fields: iter.into_iter().cloned().collect(), + } + } +} + +impl From<[StructField; N]> for StructType { + fn from(value: [StructField; N]) -> Self { + Self { + type_name: "struct".into(), + fields: value.to_vec(), + } + } +} + +impl<'a, const N: usize> From<[&'a StructField; N]> for StructType { + fn from(value: [&'a StructField; N]) -> Self { + Self { + type_name: "struct".into(), + fields: value.into_iter().cloned().collect(), + } + } +} + +impl<'a> IntoIterator for &'a StructType { + type Item = &'a StructField; + type IntoIter = std::slice::Iter<'a, StructField>; + + fn into_iter(self) -> Self::IntoIter { + self.fields.iter() + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Clone, Eq, Hash)] #[serde(rename_all = "camelCase")] /// An array stores a variable length collection of items of some type. pub struct ArrayType { @@ -350,7 +417,7 @@ impl ArrayType { } } -#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] +#[derive(Debug, Serialize, Deserialize, PartialEq, Clone, Eq, Hash)] #[serde(rename_all = "camelCase")] /// A map stores an arbitrary length collection of key-value pairs pub struct MapType { @@ -400,8 +467,8 @@ fn default_true() -> bool { true } -#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] -#[serde(rename_all = "camelCase")] +#[derive(Debug, Serialize, Deserialize, PartialEq, Clone, Eq, Hash)] +#[serde(rename_all = "snake_case")] /// Primitive types supported by Delta pub enum PrimitiveType { /// UTF-8 encoded string of characters @@ -426,25 +493,27 @@ pub enum PrimitiveType { Date, /// Microsecond precision timestamp, adjusted to UTC. Timestamp, - // TODO: timestamp without timezone + /// Micrsoecond precision timestamp with no timezone + #[serde(alias = "timestampNtz")] + TimestampNtz, #[serde( serialize_with = "serialize_decimal", deserialize_with = "deserialize_decimal", untagged )] /// Decimal: arbitrary precision decimal numbers - Decimal(i32, i32), + Decimal(u8, i8), } fn serialize_decimal( - precision: &i32, - scale: &i32, + precision: &u8, + scale: &i8, serializer: S, ) -> Result { serializer.serialize_str(&format!("decimal({},{})", precision, scale)) } -fn deserialize_decimal<'de, D>(deserializer: D) -> Result<(i32, i32), D::Error> +fn deserialize_decimal<'de, D>(deserializer: D) -> Result<(u8, i8), D::Error> where D: serde::Deserializer<'de>, { @@ -459,13 +528,13 @@ where let mut parts = str_value[8..str_value.len() - 1].split(','); let precision = parts .next() - .and_then(|part| part.trim().parse::().ok()) + .and_then(|part| part.trim().parse::().ok()) .ok_or_else(|| { serde::de::Error::custom(format!("Invalid precision in decimal: {}", str_value)) })?; let scale = parts .next() - .and_then(|part| part.trim().parse::().ok()) + .and_then(|part| part.trim().parse::().ok()) .ok_or_else(|| { serde::de::Error::custom(format!("Invalid scale in decimal: {}", str_value)) })?; @@ -487,6 +556,7 @@ impl Display for PrimitiveType { PrimitiveType::Binary => write!(f, "binary"), PrimitiveType::Date => write!(f, "date"), PrimitiveType::Timestamp => write!(f, "timestamp"), + PrimitiveType::TimestampNtz => write!(f, "timestampNtz"), PrimitiveType::Decimal(precision, scale) => { write!(f, "decimal({},{})", precision, scale) } @@ -494,9 +564,9 @@ impl Display for PrimitiveType { } } -#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] +#[derive(Debug, Serialize, Deserialize, PartialEq, Clone, Eq, Hash)] #[serde(untagged, rename_all = "camelCase")] -/// The data type of a column +/// Top level delta tdatatypes pub enum DataType { /// UTF-8 encoded string of characters Primitive(PrimitiveType), @@ -510,65 +580,45 @@ pub enum DataType { Map(Box), } -impl DataType { - /// create a new string type - pub fn string() -> Self { - DataType::Primitive(PrimitiveType::String) - } - - /// create a new long type - pub fn long() -> Self { - DataType::Primitive(PrimitiveType::Long) - } - - /// create a new integer type - pub fn integer() -> Self { - DataType::Primitive(PrimitiveType::Integer) - } - - /// create a new short type - pub fn short() -> Self { - DataType::Primitive(PrimitiveType::Short) - } - - /// create a new byte type - pub fn byte() -> Self { - DataType::Primitive(PrimitiveType::Byte) - } - - /// create a new float type - pub fn float() -> Self { - DataType::Primitive(PrimitiveType::Float) - } - - /// create a new double type - pub fn double() -> Self { - DataType::Primitive(PrimitiveType::Double) - } - - /// create a new boolean type - pub fn boolean() -> Self { - DataType::Primitive(PrimitiveType::Boolean) - } - - /// create a new binary type - pub fn binary() -> Self { - DataType::Primitive(PrimitiveType::Binary) +impl From for DataType { + fn from(map_type: MapType) -> Self { + DataType::Map(Box::new(map_type)) } +} - /// create a new date type - pub fn date() -> Self { - DataType::Primitive(PrimitiveType::Date) +impl From for DataType { + fn from(struct_type: StructType) -> Self { + DataType::Struct(Box::new(struct_type)) } +} - /// create a new timestamp type - pub fn timestamp() -> Self { - DataType::Primitive(PrimitiveType::Timestamp) +impl From for DataType { + fn from(array_type: ArrayType) -> Self { + DataType::Array(Box::new(array_type)) } +} - /// create a new decimal type - pub fn decimal(precision: usize, scale: usize) -> Self { - DataType::Primitive(PrimitiveType::Decimal(precision as i32, scale as i32)) +#[allow(missing_docs)] +impl DataType { + pub const STRING: Self = DataType::Primitive(PrimitiveType::String); + pub const LONG: Self = DataType::Primitive(PrimitiveType::Long); + pub const INTEGER: Self = DataType::Primitive(PrimitiveType::Integer); + pub const SHORT: Self = DataType::Primitive(PrimitiveType::Short); + pub const BYTE: Self = DataType::Primitive(PrimitiveType::Byte); + pub const FLOAT: Self = DataType::Primitive(PrimitiveType::Float); + pub const DOUBLE: Self = DataType::Primitive(PrimitiveType::Double); + pub const BOOLEAN: Self = DataType::Primitive(PrimitiveType::Boolean); + pub const BINARY: Self = DataType::Primitive(PrimitiveType::Binary); + pub const DATE: Self = DataType::Primitive(PrimitiveType::Date); + pub const TIMESTAMP: Self = DataType::Primitive(PrimitiveType::Timestamp); + pub const TIMESTAMPNTZ: Self = DataType::Primitive(PrimitiveType::TimestampNtz); + + pub fn decimal(precision: u8, scale: i8) -> Self { + DataType::Primitive(PrimitiveType::Decimal(precision, scale)) + } + + pub fn struct_type(fields: Vec) -> Self { + DataType::Struct(Box::new(StructType::new(fields))) } } @@ -597,6 +647,7 @@ mod tests { use super::*; use serde_json; use serde_json::json; + use std::collections::hash_map::DefaultHasher; #[test] fn test_serde_data_types() { @@ -815,4 +866,72 @@ mod tests { Invariant::new("a_map.value.element.d", "a_map.value.element.d < 4") ); } + + /// + #[test] + fn test_identity_columns() { + let buf = r#"{"type":"struct","fields":[{"name":"ID_D_DATE","type":"long","nullable":true,"metadata":{"delta.identity.start":1,"delta.identity.step":1,"delta.identity.allowExplicitInsert":false}},{"name":"TXT_DateKey","type":"string","nullable":true,"metadata":{}}]}"#; + let _schema: StructType = serde_json::from_str(buf).expect("Failed to load"); + } + + fn get_hash(field: &StructField) -> u64 { + let mut hasher = DefaultHasher::new(); + field.hash(&mut hasher); + hasher.finish() + } + + #[test] + fn test_hash_struct_field() { + // different names should result in different hashes + let field_1 = StructField::new( + "field_name_1", + DataType::Primitive(PrimitiveType::Decimal(4, 4)), + true, + ); + let field_2 = StructField::new( + "field_name_2", + DataType::Primitive(PrimitiveType::Decimal(4, 4)), + true, + ); + assert_ne!(get_hash(&field_1), get_hash(&field_2)); + + // different types should result in different hashes + let field_int = StructField::new( + "field_name", + DataType::Primitive(PrimitiveType::Integer), + true, + ); + let field_string = StructField::new( + "field_name", + DataType::Primitive(PrimitiveType::String), + true, + ); + assert_ne!(get_hash(&field_int), get_hash(&field_string)); + + // different nullability should result in different hashes + let field_true = StructField::new( + "field_name", + DataType::Primitive(PrimitiveType::Binary), + true, + ); + let field_false = StructField::new( + "field_name", + DataType::Primitive(PrimitiveType::Binary), + false, + ); + assert_ne!(get_hash(&field_true), get_hash(&field_false)); + + // case where hashes are the same + let field_1 = StructField::new( + "field_name", + DataType::Primitive(PrimitiveType::Timestamp), + true, + ); + let field_2 = StructField::new( + "field_name", + DataType::Primitive(PrimitiveType::Timestamp), + true, + ); + assert_eq!(get_hash(&field_1), get_hash(&field_2)); + } } diff --git a/crates/core/src/kernel/snapshot/log_data.rs b/crates/core/src/kernel/snapshot/log_data.rs new file mode 100644 index 0000000000..686a4110fe --- /dev/null +++ b/crates/core/src/kernel/snapshot/log_data.rs @@ -0,0 +1,741 @@ +use std::borrow::Cow; +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::{Array, Int32Array, Int64Array, MapArray, RecordBatch, StringArray, StructArray}; +use chrono::{NaiveDateTime, TimeZone, Utc}; +use indexmap::IndexMap; +use object_store::path::Path; +use object_store::ObjectMeta; +use percent_encoding::percent_decode_str; + +use crate::kernel::arrow::extract::{extract_and_cast, extract_and_cast_opt}; +use crate::kernel::{ + DataType, DeletionVectorDescriptor, Metadata, Remove, Scalar, StructField, StructType, +}; +use crate::{DeltaResult, DeltaTableError}; + +const COL_NUM_RECORDS: &str = "numRecords"; +const COL_MIN_VALUES: &str = "minValues"; +const COL_MAX_VALUES: &str = "maxValues"; +const COL_NULL_COUNT: &str = "nullCount"; + +pub(crate) type PartitionFields<'a> = Arc>; +pub(crate) type PartitionValues<'a> = IndexMap<&'a str, Scalar>; + +pub(crate) trait PartitionsExt { + fn hive_partition_path(&self) -> String; +} + +impl PartitionsExt for IndexMap<&str, Scalar> { + fn hive_partition_path(&self) -> String { + let fields = self + .iter() + .map(|(k, v)| { + let encoded = v.serialize_encoded(); + format!("{k}={encoded}") + }) + .collect::>(); + fields.join("/") + } +} + +impl PartitionsExt for IndexMap { + fn hive_partition_path(&self) -> String { + let fields = self + .iter() + .map(|(k, v)| { + let encoded = v.serialize_encoded(); + format!("{k}={encoded}") + }) + .collect::>(); + fields.join("/") + } +} + +impl PartitionsExt for Arc { + fn hive_partition_path(&self) -> String { + self.as_ref().hive_partition_path() + } +} + +/// Defines a deletion vector +#[derive(Debug, PartialEq, Clone)] +pub struct DeletionVector<'a> { + storage_type: &'a StringArray, + path_or_inline_dv: &'a StringArray, + size_in_bytes: &'a Int32Array, + cardinality: &'a Int64Array, + offset: Option<&'a Int32Array>, +} + +/// View into a deletion vector data. +#[derive(Debug)] +pub struct DeletionVectorView<'a> { + data: &'a DeletionVector<'a>, + /// Pointer to a specific row in the log data. + index: usize, +} + +impl<'a> DeletionVectorView<'a> { + /// get a unique idenitfier for the deletion vector + pub fn unique_id(&self) -> String { + if let Some(offset) = self.offset() { + format!( + "{}{}@{offset}", + self.storage_type(), + self.path_or_inline_dv() + ) + } else { + format!("{}{}", self.storage_type(), self.path_or_inline_dv()) + } + } + + fn descriptor(&self) -> DeletionVectorDescriptor { + DeletionVectorDescriptor { + storage_type: self.storage_type().parse().unwrap(), + path_or_inline_dv: self.path_or_inline_dv().to_string(), + size_in_bytes: self.size_in_bytes(), + cardinality: self.cardinality(), + offset: self.offset(), + } + } + + fn storage_type(&self) -> &str { + self.data.storage_type.value(self.index) + } + fn path_or_inline_dv(&self) -> &str { + self.data.path_or_inline_dv.value(self.index) + } + fn size_in_bytes(&self) -> i32 { + self.data.size_in_bytes.value(self.index) + } + fn cardinality(&self) -> i64 { + self.data.cardinality.value(self.index) + } + fn offset(&self) -> Option { + self.data + .offset + .and_then(|a| a.is_null(self.index).then(|| a.value(self.index))) + } +} + +/// A view into the log data representiang a single logical file. +/// +/// This stuct holds a pointer to a specific row in the log data and provides access to the +/// information stored in that row by tracking references to the underlying arrays. +/// +/// Additionally, references to some table metadata is tracked to provide higher level +/// functionality, e.g. parsing partition values. +#[derive(Debug, PartialEq)] +pub struct LogicalFile<'a> { + path: &'a StringArray, + /// The on-disk size of this data file in bytes + size: &'a Int64Array, + /// Last modification time of the file in milliseconds since the epoch. + modification_time: &'a Int64Array, + /// The partition values for this logical file. + partition_values: &'a MapArray, + /// Struct containing all available statistics for the columns in this file. + stats: &'a StructArray, + /// Array containing the deletion vector data. + deletion_vector: Option>, + + /// Pointer to a specific row in the log data. + index: usize, + /// Schema fields the table is partitioned by. + partition_fields: PartitionFields<'a>, +} + +impl LogicalFile<'_> { + /// Path to the files storage location. + pub fn path(&self) -> Cow<'_, str> { + percent_decode_str(self.path.value(self.index)).decode_utf8_lossy() + } + + /// An object store [`Path`] to the file. + /// + /// this tries to parse the file string and if that fails, it will return the string as is. + // TODO assert consisent handling of the paths encoding when reading log data so this logic can be removed. + pub fn object_store_path(&self) -> Path { + let path = self.path(); + // Try to preserve percent encoding if possible + match Path::parse(path.as_ref()) { + Ok(path) => path, + Err(_) => Path::from(path.as_ref()), + } + } + + /// File size stored on disk. + pub fn size(&self) -> i64 { + self.size.value(self.index) + } + + /// Last modification time of the file. + pub fn modification_time(&self) -> i64 { + self.modification_time.value(self.index) + } + + /// Datetime of the last modification time of the file. + pub fn modification_datetime(&self) -> DeltaResult> { + Ok(Utc.from_utc_datetime( + &NaiveDateTime::from_timestamp_millis(self.modification_time()).ok_or( + DeltaTableError::from(crate::protocol::ProtocolError::InvalidField(format!( + "invalid modification_time: {:?}", + self.modification_time() + ))), + )?, + )) + } + + /// The partition values for this logical file. + pub fn partition_values(&self) -> DeltaResult> { + if self.partition_fields.is_empty() { + return Ok(IndexMap::new()); + } + let map_value = self.partition_values.value(self.index); + let keys = map_value + .column(0) + .as_any() + .downcast_ref::() + .ok_or(DeltaTableError::Generic("()".into()))?; + let values = map_value + .column(1) + .as_any() + .downcast_ref::() + .ok_or(DeltaTableError::Generic("()".into()))?; + + let values = keys + .iter() + .zip(values.iter()) + .map(|(k, v)| { + let (key, field) = self.partition_fields.get_key_value(k.unwrap()).unwrap(); + let field_type = match field.data_type() { + DataType::Primitive(p) => Ok(p), + _ => Err(DeltaTableError::Generic( + "nested partitioning values are not supported".to_string(), + )), + }?; + Ok(( + *key, + v.map(|vv| field_type.parse_scalar(vv)) + .transpose()? + .unwrap_or(Scalar::Null(field.data_type().clone())), + )) + }) + .collect::>>()?; + + // NOTE: we recreate the map as a BTreeMap to ensure the order of the keys is consistently + // the same as the order of partition fields. + self.partition_fields + .iter() + .map(|(k, f)| { + let val = values + .get(*k) + .cloned() + .unwrap_or(Scalar::Null(f.data_type.clone())); + Ok((*k, val)) + }) + .collect::>>() + } + + /// Defines a deletion vector + pub fn deletion_vector(&self) -> Option> { + self.deletion_vector.as_ref().and_then(|arr| { + arr.storage_type + .is_valid(self.index) + .then_some(DeletionVectorView { + data: arr, + index: self.index, + }) + }) + } + + /// The number of records stored in the data file. + pub fn num_records(&self) -> Option { + self.stats + .column_by_name(COL_NUM_RECORDS) + .and_then(|c| c.as_any().downcast_ref::()) + .map(|a| a.value(self.index) as usize) + } + + /// Struct containing all available null counts for the columns in this file. + pub fn null_counts(&self) -> Option { + self.stats + .column_by_name(COL_NULL_COUNT) + .and_then(|c| Scalar::from_array(c.as_ref(), self.index)) + } + + /// Struct containing all available min values for the columns in this file. + pub fn min_values(&self) -> Option { + self.stats + .column_by_name(COL_MIN_VALUES) + .and_then(|c| Scalar::from_array(c.as_ref(), self.index)) + } + + /// Struct containing all available max values for the columns in this file. + pub fn max_values(&self) -> Option { + self.stats + .column_by_name(COL_MAX_VALUES) + .and_then(|c| Scalar::from_array(c.as_ref(), self.index)) + } + + /// Create a remove action for this logical file. + pub fn remove_action(&self, data_change: bool) -> Remove { + Remove { + // TODO use the raw (still encoded) path here once we reconciled serde ... + path: self.path().to_string(), + data_change, + deletion_timestamp: Some(Utc::now().timestamp_millis()), + extended_file_metadata: Some(true), + size: Some(self.size()), + partition_values: self.partition_values().ok().map(|pv| { + pv.iter() + .map(|(k, v)| { + ( + k.to_string(), + if v.is_null() { + None + } else { + Some(v.serialize()) + }, + ) + }) + .collect() + }), + deletion_vector: self.deletion_vector().map(|dv| dv.descriptor()), + tags: None, + base_row_id: None, + default_row_commit_version: None, + } + } +} + +impl<'a> TryFrom<&LogicalFile<'a>> for ObjectMeta { + type Error = DeltaTableError; + + fn try_from(file_stats: &LogicalFile<'a>) -> Result { + Ok(ObjectMeta { + location: file_stats.object_store_path(), + size: file_stats.size() as usize, + last_modified: file_stats.modification_datetime()?, + version: None, + e_tag: None, + }) + } +} + +/// Helper for processing data from the materialized Delta log. +pub struct FileStatsAccessor<'a> { + partition_fields: PartitionFields<'a>, + paths: &'a StringArray, + sizes: &'a Int64Array, + modification_times: &'a Int64Array, + stats: &'a StructArray, + deletion_vector: Option>, + partition_values: &'a MapArray, + length: usize, + pointer: usize, +} + +impl<'a> FileStatsAccessor<'a> { + pub(crate) fn try_new( + data: &'a RecordBatch, + metadata: &'a Metadata, + schema: &'a StructType, + ) -> DeltaResult { + let paths = extract_and_cast::(data, "add.path")?; + let sizes = extract_and_cast::(data, "add.size")?; + let modification_times = extract_and_cast::(data, "add.modificationTime")?; + let stats = extract_and_cast::(data, "add.stats_parsed")?; + let partition_values = extract_and_cast::(data, "add.partitionValues")?; + let partition_fields = Arc::new( + metadata + .partition_columns + .iter() + .map(|c| Ok((c.as_str(), schema.field_with_name(c.as_str())?))) + .collect::>>()?, + ); + let deletion_vector = extract_and_cast_opt::(data, "add.deletionVector"); + let deletion_vector = deletion_vector.and_then(|dv| { + let storage_type = extract_and_cast::(dv, "storageType").ok()?; + let path_or_inline_dv = extract_and_cast::(dv, "pathOrInlineDv").ok()?; + let size_in_bytes = extract_and_cast::(dv, "sizeInBytes").ok()?; + let cardinality = extract_and_cast::(dv, "cardinality").ok()?; + let offset = extract_and_cast_opt::(dv, "offset"); + Some(DeletionVector { + storage_type, + path_or_inline_dv, + size_in_bytes, + cardinality, + offset, + }) + }); + + Ok(Self { + partition_fields, + paths, + sizes, + modification_times, + stats, + deletion_vector, + partition_values, + length: data.num_rows(), + pointer: 0, + }) + } + + pub(crate) fn get(&self, index: usize) -> DeltaResult> { + if index >= self.length { + return Err(DeltaTableError::Generic(format!( + "index out of bounds: {} >= {}", + index, self.length + ))); + } + Ok(LogicalFile { + path: self.paths, + size: self.sizes, + modification_time: self.modification_times, + partition_values: self.partition_values, + partition_fields: self.partition_fields.clone(), + stats: self.stats, + deletion_vector: self.deletion_vector.clone(), + index, + }) + } +} + +impl<'a> Iterator for FileStatsAccessor<'a> { + type Item = LogicalFile<'a>; + + fn next(&mut self) -> Option { + if self.pointer >= self.length { + return None; + } + // Safety: we know that the pointer is within bounds + let file_stats = self.get(self.pointer).unwrap(); + self.pointer += 1; + Some(file_stats) + } +} + +/// Provides semanitc access to the log data. +/// +/// This is a helper struct that provides access to the log data in a more semantic way +/// to avid the necessiity of knowing the exact layout of the underlying log data. +pub struct LogDataHandler<'a> { + data: &'a Vec, + metadata: &'a Metadata, + schema: &'a StructType, +} + +impl<'a> LogDataHandler<'a> { + pub(crate) fn new( + data: &'a Vec, + metadata: &'a Metadata, + schema: &'a StructType, + ) -> Self { + Self { + data, + metadata, + schema, + } + } +} + +impl<'a> IntoIterator for LogDataHandler<'a> { + type Item = LogicalFile<'a>; + type IntoIter = Box + 'a>; + + fn into_iter(self) -> Self::IntoIter { + Box::new( + self.data + .iter() + .flat_map(|data| { + FileStatsAccessor::try_new(data, self.metadata, self.schema).into_iter() + }) + .flatten(), + ) + } +} + +#[cfg(feature = "datafusion")] +mod datafusion { + use std::sync::Arc; + + use arrow_arith::aggregate::sum; + use arrow_array::Int64Array; + use arrow_schema::DataType as ArrowDataType; + use datafusion_common::scalar::ScalarValue; + use datafusion_common::stats::{ColumnStatistics, Precision, Statistics}; + use datafusion_expr::AggregateFunction; + use datafusion_physical_expr::aggregate::AggregateExpr; + use datafusion_physical_expr::expressions::{Column, Max, Min}; + + use super::*; + use crate::kernel::arrow::extract::{extract_and_cast_opt, extract_column}; + + // TODO validate this works with "wide and narrow" boulds / stats + + impl FileStatsAccessor<'_> { + fn collect_count(&self, name: &str) -> Precision { + let num_records = extract_and_cast_opt::(self.stats, name); + if let Some(num_records) = num_records { + if let Some(null_count_mulls) = num_records.nulls() { + if null_count_mulls.null_count() > 0 { + Precision::Absent + } else { + sum(num_records) + .map(|s| Precision::Exact(s as usize)) + .unwrap_or(Precision::Absent) + } + } else { + sum(num_records) + .map(|s| Precision::Exact(s as usize)) + .unwrap_or(Precision::Absent) + } + } else { + Precision::Absent + } + } + + fn column_bounds( + &self, + path_step: &str, + name: &str, + fun: &AggregateFunction, + ) -> Precision { + let mut path = name.split('.'); + let array = if let Ok(array) = extract_column(self.stats, path_step, &mut path) { + array + } else { + return Precision::Absent; + }; + + if array.data_type().is_primitive() { + let agg: Box = match fun { + AggregateFunction::Min => Box::new(Min::new( + // NOTE: this is just a placeholder, we never evalutae this expression + Arc::new(Column::new(name, 0)), + name, + array.data_type().clone(), + )), + AggregateFunction::Max => Box::new(Max::new( + // NOTE: this is just a placeholder, we never evalutae this expression + Arc::new(Column::new(name, 0)), + name, + array.data_type().clone(), + )), + _ => return Precision::Absent, + }; + let mut accum = agg.create_accumulator().ok().unwrap(); + return accum + .update_batch(&[array.clone()]) + .ok() + .and_then(|_| accum.evaluate().ok()) + .map(Precision::Exact) + .unwrap_or(Precision::Absent); + } + + match array.data_type() { + ArrowDataType::Struct(fields) => { + return fields + .iter() + .map(|f| { + self.column_bounds(path_step, &format!("{name}.{}", f.name()), fun) + }) + .map(|s| match s { + Precision::Exact(s) => Some(s), + _ => None, + }) + .collect::>>() + .map(|o| Precision::Exact(ScalarValue::Struct(Some(o), fields.clone()))) + .unwrap_or(Precision::Absent); + } + _ => Precision::Absent, + } + } + + fn num_records(&self) -> Precision { + self.collect_count(COL_NUM_RECORDS) + } + + fn total_size_files(&self) -> Precision { + let size = self + .sizes + .iter() + .flat_map(|s| s.map(|s| s as usize)) + .sum::(); + Precision::Inexact(size) + } + + fn column_stats(&self, name: impl AsRef) -> DeltaResult { + let null_count_col = format!("{COL_NULL_COUNT}.{}", name.as_ref()); + let null_count = self.collect_count(&null_count_col); + + let min_value = + self.column_bounds(COL_MIN_VALUES, name.as_ref(), &AggregateFunction::Min); + let min_value = match &min_value { + Precision::Exact(value) if value.is_null() => Precision::Absent, + // TODO this is a hack, we should not be casting here but rather when we read the checkpoint data. + // it seems sometimes the min/max values are stored as nanoseconds and sometimes as microseconds? + Precision::Exact(ScalarValue::TimestampNanosecond(a, b)) => Precision::Exact( + ScalarValue::TimestampMicrosecond(a.map(|v| v / 1000), b.clone()), + ), + _ => min_value, + }; + + let max_value = + self.column_bounds(COL_MAX_VALUES, name.as_ref(), &AggregateFunction::Max); + let max_value = match &max_value { + Precision::Exact(value) if value.is_null() => Precision::Absent, + Precision::Exact(ScalarValue::TimestampNanosecond(a, b)) => Precision::Exact( + ScalarValue::TimestampMicrosecond(a.map(|v| v / 1000), b.clone()), + ), + _ => max_value, + }; + + Ok(ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count: Precision::Absent, + }) + } + } + + trait StatsExt { + fn add(&self, other: &Self) -> Self; + } + + impl StatsExt for ColumnStatistics { + fn add(&self, other: &Self) -> Self { + Self { + null_count: self.null_count.add(&other.null_count), + max_value: self.max_value.max(&other.max_value), + min_value: self.min_value.min(&other.min_value), + distinct_count: self.distinct_count.add(&other.distinct_count), + } + } + } + + impl LogDataHandler<'_> { + fn num_records(&self) -> Precision { + self.data + .iter() + .flat_map(|b| { + FileStatsAccessor::try_new(b, self.metadata, self.schema) + .map(|a| a.num_records()) + }) + .reduce(|acc, num_records| acc.add(&num_records)) + .unwrap_or(Precision::Absent) + } + + fn total_size_files(&self) -> Precision { + self.data + .iter() + .flat_map(|b| { + FileStatsAccessor::try_new(b, self.metadata, self.schema) + .map(|a| a.total_size_files()) + }) + .reduce(|acc, size| acc.add(&size)) + .unwrap_or(Precision::Absent) + } + + pub(crate) fn column_stats(&self, name: impl AsRef) -> Option { + self.data + .iter() + .flat_map(|b| { + FileStatsAccessor::try_new(b, self.metadata, self.schema) + .map(|a| a.column_stats(name.as_ref())) + }) + .collect::, _>>() + .ok()? + .iter() + .fold(None::, |acc, stats| match (acc, stats) { + (None, stats) => Some(stats.clone()), + (Some(acc), stats) => Some(acc.add(stats)), + }) + } + + pub(crate) fn statistics(&self) -> Option { + let num_rows = self.num_records(); + let total_byte_size = self.total_size_files(); + let column_statistics = self + .schema + .fields() + .iter() + .map(|f| self.column_stats(f.name())) + .collect::>>()?; + Some(Statistics { + num_rows, + total_byte_size, + column_statistics, + }) + } + } +} + +#[cfg(all(test, feature = "datafusion"))] +mod tests { + + #[tokio::test] + async fn read_delta_1_2_1_struct_stats_table() { + let table_uri = "../test/tests/data/delta-1.2.1-only-struct-stats"; + let table_from_struct_stats = crate::open_table(table_uri).await.unwrap(); + let table_from_json_stats = crate::open_table_with_version(table_uri, 1).await.unwrap(); + + let json_action = table_from_json_stats + .snapshot() + .unwrap() + .snapshot + .files() + .find(|f| { + f.path().ends_with( + "part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet", + ) + }) + .unwrap(); + + let struct_action = table_from_struct_stats + .snapshot() + .unwrap() + .snapshot + .files() + .find(|f| { + f.path().ends_with( + "part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet", + ) + }) + .unwrap(); + + assert_eq!(json_action.path(), struct_action.path()); + assert_eq!( + json_action.partition_values().unwrap(), + struct_action.partition_values().unwrap() + ); + // assert_eq!( + // json_action.max_values().unwrap(), + // struct_action.max_values().unwrap() + // ); + // assert_eq!( + // json_action.min_values().unwrap(), + // struct_action.min_values().unwrap() + // ); + } + + #[tokio::test] + async fn df_stats_delta_1_2_1_struct_stats_table() { + let table_uri = "../test/tests/data/delta-1.2.1-only-struct-stats"; + let table_from_struct_stats = crate::open_table(table_uri).await.unwrap(); + + let file_stats = table_from_struct_stats + .snapshot() + .unwrap() + .snapshot + .log_data(); + + let col_stats = file_stats.statistics(); + println!("{:?}", col_stats); + } +} diff --git a/crates/core/src/kernel/snapshot/log_segment.rs b/crates/core/src/kernel/snapshot/log_segment.rs new file mode 100644 index 0000000000..0b82231ee8 --- /dev/null +++ b/crates/core/src/kernel/snapshot/log_segment.rs @@ -0,0 +1,633 @@ +use std::cmp::Ordering; +use std::collections::VecDeque; +use std::sync::Arc; + +use arrow_array::RecordBatch; +use chrono::Utc; +use futures::{stream::BoxStream, StreamExt, TryStreamExt}; +use itertools::Itertools; +use lazy_static::lazy_static; +use object_store::path::Path; +use object_store::{Error as ObjectStoreError, ObjectMeta, ObjectStore}; +use parquet::arrow::arrow_reader::ArrowReaderOptions; +use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; +use regex::Regex; +use serde::{Deserialize, Serialize}; +use tracing::debug; + +use super::parse; +use crate::kernel::{arrow::json, ActionType, Metadata, Protocol, Schema, StructType}; +use crate::logstore::LogStore; +use crate::operations::transaction::CommitData; +use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; + +const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; + +lazy_static! { + static ref CHECKPOINT_FILE_PATTERN: Regex = + Regex::new(r"\d+\.checkpoint(\.\d+\.\d+)?\.parquet").unwrap(); + static ref DELTA_FILE_PATTERN: Regex = Regex::new(r"^\d+\.json$").unwrap(); + pub(super) static ref COMMIT_SCHEMA: StructType = StructType::new(vec![ + ActionType::Add.schema_field().clone(), + ActionType::Remove.schema_field().clone(), + ]); + pub(super) static ref CHECKPOINT_SCHEMA: StructType = + StructType::new(vec![ActionType::Add.schema_field().clone(),]); + pub(super) static ref TOMBSTONE_SCHEMA: StructType = + StructType::new(vec![ActionType::Remove.schema_field().clone(),]); +} + +/// Trait to extend a file path representation with delta specific functionality +/// +/// specifically, this trait adds the ability to recognize valid log files and +/// parse the version number from a log file path +// TODO handle compaction files +pub(super) trait PathExt { + fn child(&self, path: impl AsRef) -> DeltaResult; + /// Returns the last path segment if not terminated with a "/" + fn filename(&self) -> Option<&str>; + + /// Parse the version number assuming a commit json or checkpoint parquet file + fn commit_version(&self) -> Option { + self.filename() + .and_then(|f| f.split_once('.')) + .and_then(|(name, _)| name.parse().ok()) + } + + /// Returns true if the file is a checkpoint parquet file + fn is_checkpoint_file(&self) -> bool { + self.filename() + .map(|name| CHECKPOINT_FILE_PATTERN.captures(name).is_some()) + .unwrap_or(false) + } + + /// Returns true if the file is a commit json file + fn is_commit_file(&self) -> bool { + self.filename() + .map(|name| DELTA_FILE_PATTERN.captures(name).is_some()) + .unwrap_or(false) + } +} + +impl PathExt for Path { + fn child(&self, path: impl AsRef) -> DeltaResult { + Ok(self.child(path.as_ref())) + } + + fn filename(&self) -> Option<&str> { + self.filename() + } +} + +#[derive(Debug, Clone, PartialEq)] +pub(super) struct LogSegment { + pub(super) version: i64, + pub(super) commit_files: VecDeque, + pub(super) checkpoint_files: Vec, +} + +impl LogSegment { + /// Try to create a new [`LogSegment`] + /// + /// This will list the entire log directory and find all relevant files for the given table version. + pub async fn try_new( + table_root: &Path, + version: Option, + store: &dyn ObjectStore, + ) -> DeltaResult { + let log_url = table_root.child("_delta_log"); + let maybe_cp = read_last_checkpoint(store, &log_url).await?; + + // List relevant files from log + let (mut commit_files, checkpoint_files) = match (maybe_cp, version) { + (Some(cp), None) => list_log_files_with_checkpoint(&cp, store, &log_url).await?, + (Some(cp), Some(v)) if cp.version <= v => { + list_log_files_with_checkpoint(&cp, store, &log_url).await? + } + _ => list_log_files(store, &log_url, version, None).await?, + }; + + // remove all files above requested version + if let Some(version) = version { + commit_files.retain(|meta| meta.location.commit_version() <= Some(version)); + } + + let mut segment = Self { + version: 0, + commit_files: commit_files.into(), + checkpoint_files, + }; + if segment.commit_files.is_empty() && segment.checkpoint_files.is_empty() { + return Err(DeltaTableError::NotATable("no log files".into())); + } + // get the effective version from chosen files + let version_eff = segment.file_version().ok_or(DeltaTableError::Generic( + "failed to get effective version".into(), + ))?; // TODO: A more descriptive error + segment.version = version_eff; + segment.validate()?; + + if let Some(v) = version { + if version_eff != v { + // TODO more descriptive error + return Err(DeltaTableError::Generic("missing version".into())); + } + } + + Ok(segment) + } + + /// Try to create a new [`LogSegment`] from a slice of the log. + /// + /// Ths will create a new [`LogSegment`] from the log with all relevant log files + /// starting at `start_version` and ending at `end_version`. + pub async fn try_new_slice( + table_root: &Path, + start_version: i64, + end_version: Option, + log_store: &dyn LogStore, + ) -> DeltaResult { + debug!( + "try_new_slice: start_version: {}, end_version: {:?}", + start_version, end_version + ); + log_store.refresh().await?; + let log_url = table_root.child("_delta_log"); + let (mut commit_files, checkpoint_files) = list_log_files( + log_store.object_store().as_ref(), + &log_url, + end_version, + Some(start_version), + ) + .await?; + // remove all files above requested version + if let Some(version) = end_version { + commit_files.retain(|meta| meta.location.commit_version() <= Some(version)); + } + let mut segment = Self { + version: start_version, + commit_files: commit_files.into(), + checkpoint_files, + }; + segment.version = segment + .file_version() + .unwrap_or(end_version.unwrap_or(start_version)); + Ok(segment) + } + + pub fn validate(&self) -> DeltaResult<()> { + let checkpoint_version = self + .checkpoint_files + .iter() + .filter_map(|f| f.location.commit_version()) + .max(); + if let Some(v) = checkpoint_version { + if !self + .commit_files + .iter() + .all(|f| f.location.commit_version() > Some(v)) + { + return Err(DeltaTableError::Generic("inconsistent log segment".into())); + } + } + Ok(()) + } + + /// Returns the highes commit version number in the log segment + pub fn file_version(&self) -> Option { + self.commit_files + .iter() + .filter_map(|f| f.location.commit_version()) + .max() + .or(self + .checkpoint_files + .first() + .and_then(|f| f.location.commit_version())) + } + + #[cfg(test)] + pub(super) fn new_test<'a>( + commits: impl IntoIterator, + ) -> DeltaResult<(Self, Vec>)> { + let mut log = Self { + version: -1, + commit_files: Default::default(), + checkpoint_files: Default::default(), + }; + let iter = log + .advance( + commits, + &Path::default(), + crate::kernel::models::fields::log_schema(), + &Default::default(), + )? + .collect_vec(); + Ok((log, iter)) + } + + pub fn version(&self) -> i64 { + self.version + } + + /// Returns the last modified timestamp for a commit file with the given version + pub fn version_timestamp(&self, version: i64) -> Option> { + self.commit_files + .iter() + .find(|f| f.location.commit_version() == Some(version)) + .map(|f| f.last_modified) + } + + pub(super) fn commit_stream( + &self, + store: Arc, + read_schema: &Schema, + config: &DeltaTableConfig, + ) -> DeltaResult>> { + let decoder = json::get_decoder(Arc::new(read_schema.try_into()?), config)?; + let stream = futures::stream::iter(self.commit_files.iter()) + .map(move |meta| { + let store = store.clone(); + async move { store.get(&meta.location).await?.bytes().await } + }) + .buffered(config.log_buffer_size); + Ok(json::decode_stream(decoder, stream).boxed()) + } + + pub(super) fn checkpoint_stream( + &self, + store: Arc, + _read_schema: &Schema, + config: &DeltaTableConfig, + ) -> BoxStream<'_, DeltaResult> { + let batch_size = config.log_batch_size; + futures::stream::iter(self.checkpoint_files.clone()) + .map(move |meta| { + let store = store.clone(); + async move { + let reader = ParquetObjectReader::new(store, meta); + let options = ArrowReaderOptions::new(); //.with_page_index(enable_page_index); + let builder = + ParquetRecordBatchStreamBuilder::new_with_options(reader, options).await?; + builder.with_batch_size(batch_size).build() + } + }) + .buffered(config.log_buffer_size) + .try_flatten() + .map_err(Into::into) + .boxed() + } + + /// Read [`Protocol`] and [`Metadata`] actions + pub(super) async fn read_metadata( + &self, + store: Arc, + config: &DeltaTableConfig, + ) -> DeltaResult<(Option, Option)> { + lazy_static::lazy_static! { + static ref READ_SCHEMA: StructType = StructType::new(vec![ + ActionType::Protocol.schema_field().clone(), + ActionType::Metadata.schema_field().clone(), + ]); + } + + let mut maybe_protocol = None; + let mut maybe_metadata = None; + + let mut commit_stream = self.commit_stream(store.clone(), &READ_SCHEMA, config)?; + while let Some(batch) = commit_stream.next().await { + let batch = batch?; + if maybe_protocol.is_none() { + if let Some(p) = parse::read_protocol(&batch)? { + maybe_protocol.replace(p); + }; + } + if maybe_metadata.is_none() { + if let Some(m) = parse::read_metadata(&batch)? { + maybe_metadata.replace(m); + }; + } + if maybe_protocol.is_some() && maybe_metadata.is_some() { + return Ok((maybe_protocol, maybe_metadata)); + } + } + + let mut checkpoint_stream = self.checkpoint_stream(store.clone(), &READ_SCHEMA, config); + while let Some(batch) = checkpoint_stream.next().await { + let batch = batch?; + if maybe_protocol.is_none() { + if let Some(p) = parse::read_protocol(&batch)? { + maybe_protocol.replace(p); + }; + } + if maybe_metadata.is_none() { + if let Some(m) = parse::read_metadata(&batch)? { + maybe_metadata.replace(m); + }; + } + if maybe_protocol.is_some() && maybe_metadata.is_some() { + return Ok((maybe_protocol, maybe_metadata)); + } + } + + Ok((maybe_protocol, maybe_metadata)) + } + + /// Advance the log segment with new commits + /// + /// Returns an iterator over record batches, as if the commits were read from the log. + /// The input commits should be in order in which they would be commited to the table. + pub(super) fn advance<'a>( + &mut self, + commits: impl IntoIterator, + table_root: &Path, + read_schema: &Schema, + config: &DeltaTableConfig, + ) -> DeltaResult> + '_> { + let log_path = table_root.child("_delta_log"); + let mut decoder = json::get_decoder(Arc::new(read_schema.try_into()?), config)?; + + let mut commit_data = Vec::new(); + for commit in commits { + self.version += 1; + let path = log_path.child(format!("{:020}.json", self.version)); + let bytes = commit.get_bytes()?; + let meta = ObjectMeta { + location: path, + size: bytes.len(), + last_modified: Utc::now(), + e_tag: None, + version: None, + }; + // NOTE: We always assume the commit files are sorted in reverse order + self.commit_files.push_front(meta); + let reader = json::get_reader(&bytes); + let batches = + json::decode_reader(&mut decoder, reader).collect::, _>>()?; + commit_data.push(batches); + } + + // NOTE: Most recent commits need to be processed first + commit_data.reverse(); + Ok(commit_data.into_iter().flatten().map(Ok)) + } +} + +#[derive(Debug, Deserialize, Serialize)] +#[serde(rename_all = "camelCase")] +struct CheckpointMetadata { + /// The version of the table when the last checkpoint was made. + #[allow(unreachable_pub)] // used by acceptance tests (TODO make an fn accessor?) + pub version: i64, + /// The number of actions that are stored in the checkpoint. + pub(crate) size: i32, + /// The number of fragments if the last checkpoint was written in multiple parts. + pub(crate) parts: Option, + /// The number of bytes of the checkpoint. + pub(crate) size_in_bytes: Option, + /// The number of AddFile actions in the checkpoint. + pub(crate) num_of_add_files: Option, + /// The schema of the checkpoint file. + pub(crate) checkpoint_schema: Option, + /// The checksum of the last checkpoint JSON. + pub(crate) checksum: Option, +} + +/// Try reading the `_last_checkpoint` file. +/// +/// In case the file is not found, `None` is returned. +async fn read_last_checkpoint( + fs_client: &dyn ObjectStore, + log_root: &Path, +) -> DeltaResult> { + let file_path = log_root.child(LAST_CHECKPOINT_FILE_NAME); + match fs_client.get(&file_path).await { + Ok(data) => { + let data = data.bytes().await?; + Ok(Some(serde_json::from_slice(&data)?)) + } + Err(ObjectStoreError::NotFound { .. }) => Ok(None), + Err(err) => Err(err.into()), + } +} + +/// List all log files after a given checkpoint. +async fn list_log_files_with_checkpoint( + cp: &CheckpointMetadata, + fs_client: &dyn ObjectStore, + log_root: &Path, +) -> DeltaResult<(Vec, Vec)> { + let version_prefix = format!("{:020}", cp.version); + let start_from = log_root.child(version_prefix.as_str()); + + let files = fs_client + .list_with_offset(Some(log_root), &start_from) + .try_collect::>() + .await? + .into_iter() + // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? + .filter(|f| f.location.commit_version().is_some()) + .collect::>(); + + let mut commit_files = files + .iter() + .filter_map(|f| { + if f.location.is_commit_file() && f.location.commit_version() > Some(cp.version) { + Some(f.clone()) + } else { + None + } + }) + .collect_vec(); + + // NOTE: this will sort in reverse order + commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + + let checkpoint_files = files + .iter() + .filter_map(|f| { + if f.location.is_checkpoint_file() && f.location.commit_version() == Some(cp.version) { + Some(f.clone()) + } else { + None + } + }) + .collect_vec(); + + // TODO raise a proper error + assert_eq!(checkpoint_files.len(), cp.parts.unwrap_or(1) as usize); + + Ok((commit_files, checkpoint_files)) +} + +/// List relevant log files. +/// +/// Relevant files are the max checkpoint found and all subsequent commits. +pub(super) async fn list_log_files( + fs_client: &dyn ObjectStore, + log_root: &Path, + max_version: Option, + start_version: Option, +) -> DeltaResult<(Vec, Vec)> { + let max_version = max_version.unwrap_or(i64::MAX - 1); + let start_from = log_root.child(format!("{:020}", start_version.unwrap_or(0)).as_str()); + + let mut max_checkpoint_version = -1_i64; + let mut commit_files = Vec::with_capacity(25); + let mut checkpoint_files = Vec::with_capacity(10); + + for meta in fs_client + .list_with_offset(Some(log_root), &start_from) + .try_collect::>() + .await? + { + if meta.location.commit_version().unwrap_or(i64::MAX) <= max_version + && meta.location.commit_version() >= start_version + { + if meta.location.is_checkpoint_file() { + let version = meta.location.commit_version().unwrap_or(0); + match version.cmp(&max_checkpoint_version) { + Ordering::Greater => { + max_checkpoint_version = version; + checkpoint_files.clear(); + checkpoint_files.push(meta); + } + Ordering::Equal => { + checkpoint_files.push(meta); + } + _ => {} + } + } else if meta.location.is_commit_file() { + commit_files.push(meta); + } + } + } + + commit_files.retain(|f| f.location.commit_version().unwrap_or(0) > max_checkpoint_version); + // NOTE this will sort in reverse order + commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + + Ok((commit_files, checkpoint_files)) +} + +#[cfg(test)] +pub(super) mod tests { + use deltalake_test::utils::*; + + use super::*; + + pub(crate) async fn test_log_segment(context: &IntegrationContext) -> TestResult { + read_log_files(context).await?; + read_metadata(context).await?; + log_segment_serde(context).await?; + + Ok(()) + } + + async fn log_segment_serde(context: &IntegrationContext) -> TestResult { + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + + let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let bytes = serde_json::to_vec(&segment).unwrap(); + let actual: LogSegment = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual.version(), segment.version()); + assert_eq!(actual.commit_files.len(), segment.commit_files.len()); + assert_eq!( + actual.checkpoint_files.len(), + segment.checkpoint_files.len() + ); + + Ok(()) + } + + async fn read_log_files(context: &IntegrationContext) -> TestResult { + let store = context + .table_builder(TestTables::SimpleWithCheckpoint) + .build_storage()? + .object_store(); + + let log_path = Path::from("_delta_log"); + let cp = read_last_checkpoint(store.as_ref(), &log_path) + .await? + .unwrap(); + assert_eq!(cp.version, 10); + + let (log, check) = list_log_files_with_checkpoint(&cp, store.as_ref(), &log_path).await?; + assert_eq!(log.len(), 0); + assert_eq!(check.len(), 1); + + let (log, check) = list_log_files(store.as_ref(), &log_path, None, None).await?; + assert_eq!(log.len(), 0); + assert_eq!(check.len(), 1); + + let (log, check) = list_log_files(store.as_ref(), &log_path, Some(8), None).await?; + assert_eq!(log.len(), 9); + assert_eq!(check.len(), 0); + + let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + assert_eq!(segment.version, 10); + assert_eq!(segment.commit_files.len(), 0); + assert_eq!(segment.checkpoint_files.len(), 1); + + let segment = LogSegment::try_new(&Path::default(), Some(8), store.as_ref()).await?; + assert_eq!(segment.version, 8); + assert_eq!(segment.commit_files.len(), 9); + assert_eq!(segment.checkpoint_files.len(), 0); + + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + + let (log, check) = list_log_files(store.as_ref(), &log_path, None, None).await?; + assert_eq!(log.len(), 5); + assert_eq!(check.len(), 0); + + let (log, check) = list_log_files(store.as_ref(), &log_path, Some(2), None).await?; + assert_eq!(log.len(), 3); + assert_eq!(check.len(), 0); + + Ok(()) + } + + async fn read_metadata(context: &IntegrationContext) -> TestResult { + let store = context + .table_builder(TestTables::WithDvSmall) + .build_storage()? + .object_store(); + let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let (protocol, _metadata) = segment + .read_metadata(store.clone(), &Default::default()) + .await?; + let protocol = protocol.unwrap(); + + let expected = Protocol { + min_reader_version: 3, + min_writer_version: 7, + reader_features: Some(vec!["deletionVectors".into()].into_iter().collect()), + writer_features: Some(vec!["deletionVectors".into()].into_iter().collect()), + }; + assert_eq!(protocol, expected); + + Ok(()) + } + + #[test] + pub fn is_commit_file_only_matches_commits() { + for path in [0, 1, 5, 10, 100, i64::MAX] + .into_iter() + .map(crate::storage::commit_uri_from_version) + { + assert!(path.is_commit_file()); + } + + let not_commits = ["_delta_log/_commit_2132c4fe-4077-476c-b8f5-e77fea04f170.json.tmp"]; + + for not_commit in not_commits { + let path = Path::from(not_commit); + assert!(!path.is_commit_file()); + } + } +} diff --git a/crates/core/src/kernel/snapshot/mod.rs b/crates/core/src/kernel/snapshot/mod.rs new file mode 100644 index 0000000000..e27fe3e2f0 --- /dev/null +++ b/crates/core/src/kernel/snapshot/mod.rs @@ -0,0 +1,778 @@ +//! Delta table snapshots +//! +//! A snapshot represents the state of a Delta Table at a given version. +//! +//! There are two types of snapshots: +//! +//! - [`Snapshot`] is a snapshot where most data is loaded on demand and only the +//! bare minimum - [`Protocol`] and [`Metadata`] - is cached in memory. +//! - [`EagerSnapshot`] is a snapshot where much more log data is eagerly loaded into memory. +//! +//! The sub modules provide structures and methods that aid in generating +//! and consuming snapshots. +//! +//! ## Reading the log +//! +//! + +use std::io::{BufRead, BufReader, Cursor}; +use std::sync::Arc; + +use ::serde::{Deserialize, Serialize}; +use arrow_array::RecordBatch; +use futures::stream::BoxStream; +use futures::{StreamExt, TryStreamExt}; +use object_store::path::Path; +use object_store::ObjectStore; + +use self::log_segment::{LogSegment, PathExt}; +use self::parse::{read_adds, read_removes}; +use self::replay::{LogMapper, LogReplayScanner, ReplayStream}; +use super::{Action, Add, CommitInfo, DataType, Metadata, Protocol, Remove, StructField}; +use crate::kernel::StructType; +use crate::logstore::LogStore; +use crate::operations::transaction::CommitData; +use crate::table::config::TableConfig; +use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; + +mod log_data; +mod log_segment; +pub(crate) mod parse; +mod replay; +mod serde; + +pub use log_data::*; + +/// A snapshot of a Delta table +#[derive(Debug, Serialize, Deserialize, Clone, PartialEq)] +pub struct Snapshot { + log_segment: LogSegment, + config: DeltaTableConfig, + protocol: Protocol, + metadata: Metadata, + schema: StructType, + // TODO make this an URL + /// path of the table root within the object store + table_url: String, +} + +impl Snapshot { + /// Create a new [`Snapshot`] instance + pub async fn try_new( + table_root: &Path, + store: Arc, + config: DeltaTableConfig, + version: Option, + ) -> DeltaResult { + let log_segment = LogSegment::try_new(table_root, version, store.as_ref()).await?; + let (protocol, metadata) = log_segment.read_metadata(store.clone(), &config).await?; + if metadata.is_none() || protocol.is_none() { + return Err(DeltaTableError::Generic( + "Cannot read metadata from log segment".into(), + )); + }; + let (metadata, protocol) = (metadata.unwrap(), protocol.unwrap()); + let schema = serde_json::from_str(&metadata.schema_string)?; + Ok(Self { + log_segment, + config, + protocol, + metadata, + schema, + table_url: table_root.to_string(), + }) + } + + #[cfg(test)] + pub fn new_test<'a>( + commits: impl IntoIterator, + ) -> DeltaResult<(Self, RecordBatch)> { + use arrow_select::concat::concat_batches; + let (log_segment, batches) = LogSegment::new_test(commits)?; + let batch = batches.into_iter().collect::, _>>()?; + let batch = concat_batches(&batch[0].schema(), &batch)?; + let protocol = parse::read_protocol(&batch)?.unwrap(); + let metadata = parse::read_metadata(&batch)?.unwrap(); + let schema = serde_json::from_str(&metadata.schema_string)?; + Ok(( + Self { + log_segment, + config: Default::default(), + protocol, + metadata, + schema, + table_url: Path::default().to_string(), + }, + batch, + )) + } + + /// Update the snapshot to the given version + pub async fn update( + &mut self, + log_store: Arc, + target_version: Option, + ) -> DeltaResult<()> { + self.update_inner(log_store, target_version).await?; + Ok(()) + } + + async fn update_inner( + &mut self, + log_store: Arc, + target_version: Option, + ) -> DeltaResult> { + if let Some(version) = target_version { + if version == self.version() { + return Ok(None); + } + if version < self.version() { + return Err(DeltaTableError::Generic("Cannot downgrade snapshot".into())); + } + } + let log_segment = LogSegment::try_new_slice( + &Path::default(), + self.version() + 1, + target_version, + log_store.as_ref(), + ) + .await?; + if log_segment.commit_files.is_empty() && log_segment.checkpoint_files.is_empty() { + return Ok(None); + } + + let (protocol, metadata) = log_segment + .read_metadata(log_store.object_store().clone(), &self.config) + .await?; + if let Some(protocol) = protocol { + self.protocol = protocol; + } + if let Some(metadata) = metadata { + self.metadata = metadata; + self.schema = serde_json::from_str(&self.metadata.schema_string)?; + } + + if !log_segment.checkpoint_files.is_empty() { + self.log_segment.checkpoint_files = log_segment.checkpoint_files.clone(); + self.log_segment.commit_files = log_segment.commit_files.clone(); + } else { + for file in &log_segment.commit_files { + self.log_segment.commit_files.push_front(file.clone()); + } + } + + self.log_segment.version = log_segment.version; + + Ok(Some(log_segment)) + } + + /// Get the table version of the snapshot + pub fn version(&self) -> i64 { + self.log_segment.version() + } + + /// Get the table schema of the snapshot + pub fn schema(&self) -> &StructType { + &self.schema + } + + /// Get the table metadata of the snapshot + pub fn metadata(&self) -> &Metadata { + &self.metadata + } + + /// Get the table protocol of the snapshot + pub fn protocol(&self) -> &Protocol { + &self.protocol + } + + /// Get the table root of the snapshot + pub fn table_root(&self) -> Path { + Path::from(self.table_url.clone()) + } + + /// Well known table configuration + pub fn table_config(&self) -> TableConfig<'_> { + TableConfig(&self.metadata.configuration) + } + + /// Get the files in the snapshot + pub fn files( + &self, + store: Arc, + ) -> DeltaResult>>> { + let log_stream = self.log_segment.commit_stream( + store.clone(), + &log_segment::COMMIT_SCHEMA, + &self.config, + )?; + let checkpoint_stream = self.log_segment.checkpoint_stream( + store, + &log_segment::CHECKPOINT_SCHEMA, + &self.config, + ); + ReplayStream::try_new(log_stream, checkpoint_stream, self) + } + + /// Get the commit infos in the snapshot + pub(crate) async fn commit_infos( + &self, + store: Arc, + limit: Option, + ) -> DeltaResult>>> { + let log_root = self.table_root().child("_delta_log"); + let start_from = log_root.child( + format!( + "{:020}", + limit + .map(|l| (self.version() - l as i64 + 1).max(0)) + .unwrap_or(0) + ) + .as_str(), + ); + + let mut commit_files = Vec::new(); + for meta in store + .list_with_offset(Some(&log_root), &start_from) + .try_collect::>() + .await? + { + if meta.location.is_commit_file() { + commit_files.push(meta); + } + } + commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + Ok(futures::stream::iter(commit_files) + .map(move |meta| { + let store = store.clone(); + async move { + let commit_log_bytes = store.get(&meta.location).await?.bytes().await?; + let reader = BufReader::new(Cursor::new(commit_log_bytes)); + for line in reader.lines() { + let action: Action = serde_json::from_str(line?.as_str())?; + if let Action::CommitInfo(commit_info) = action { + return Ok::<_, DeltaTableError>(Some(commit_info)); + } + } + Ok(None) + } + }) + .buffered(self.config.log_buffer_size) + .boxed()) + } + + pub(crate) fn tombstones( + &self, + store: Arc, + ) -> DeltaResult>>> { + let log_stream = self.log_segment.commit_stream( + store.clone(), + &log_segment::TOMBSTONE_SCHEMA, + &self.config, + )?; + let checkpoint_stream = + self.log_segment + .checkpoint_stream(store, &log_segment::TOMBSTONE_SCHEMA, &self.config); + + Ok(log_stream + .chain(checkpoint_stream) + .map(|batch| match batch { + Ok(batch) => read_removes(&batch), + Err(e) => Err(e), + }) + .boxed()) + } + + /// Get the statistics schema of the snapshot + pub fn stats_schema(&self) -> DeltaResult { + let stats_fields = if let Some(stats_cols) = self.table_config().stats_columns() { + stats_cols + .iter() + .map(|col| match self.schema().field_with_name(col) { + Ok(field) => match field.data_type() { + DataType::Map(_) | DataType::Array(_) | &DataType::BINARY => { + Err(DeltaTableError::Generic(format!( + "Stats column {} has unsupported type {}", + col, + field.data_type() + ))) + } + _ => Ok(StructField::new( + field.name(), + field.data_type().clone(), + true, + )), + }, + _ => Err(DeltaTableError::Generic(format!( + "Stats column {} not found in schema", + col + ))), + }) + .collect::, _>>()? + } else { + let num_indexed_cols = self.table_config().num_indexed_cols(); + self.schema() + .fields + .iter() + .enumerate() + .filter_map(|(idx, f)| match f.data_type() { + DataType::Map(_) | DataType::Array(_) | &DataType::BINARY => None, + _ if num_indexed_cols < 0 || (idx as i32) < num_indexed_cols => { + Some(StructField::new(f.name(), f.data_type().clone(), true)) + } + _ => None, + }) + .collect() + }; + Ok(StructType::new(vec![ + StructField::new("numRecords", DataType::LONG, true), + StructField::new("minValues", StructType::new(stats_fields.clone()), true), + StructField::new("maxValues", StructType::new(stats_fields.clone()), true), + StructField::new( + "nullCount", + StructType::new(stats_fields.iter().filter_map(to_count_field).collect()), + true, + ), + ])) + } +} + +/// A snapshot of a Delta table that has been eagerly loaded into memory. +#[derive(Debug, Clone, PartialEq)] +pub struct EagerSnapshot { + snapshot: Snapshot, + // NOTE: this is a Vec of RecordBatch instead of a single RecordBatch because + // we do not yet enforce a consistent schema across all batches we read from the log. + files: Vec, +} + +impl EagerSnapshot { + /// Create a new [`EagerSnapshot`] instance + pub async fn try_new( + table_root: &Path, + store: Arc, + config: DeltaTableConfig, + version: Option, + ) -> DeltaResult { + let snapshot = Snapshot::try_new(table_root, store.clone(), config, version).await?; + let files = snapshot.files(store)?.try_collect().await?; + Ok(Self { snapshot, files }) + } + + #[cfg(test)] + pub fn new_test<'a>(commits: impl IntoIterator) -> DeltaResult { + let (snapshot, batch) = Snapshot::new_test(commits)?; + let mut files = Vec::new(); + let mut scanner = LogReplayScanner::new(); + files.push(scanner.process_files_batch(&batch, true)?); + let mapper = LogMapper::try_new(&snapshot)?; + files = files + .into_iter() + .map(|b| mapper.map_batch(b)) + .collect::>>()?; + Ok(Self { snapshot, files }) + } + + /// Update the snapshot to the given version + pub async fn update( + &mut self, + log_store: Arc, + target_version: Option, + ) -> DeltaResult<()> { + if Some(self.version()) == target_version { + return Ok(()); + } + let new_slice = self + .snapshot + .update_inner(log_store.clone(), target_version) + .await?; + if let Some(new_slice) = new_slice { + let files = std::mem::take(&mut self.files); + let log_stream = new_slice.commit_stream( + log_store.object_store().clone(), + &log_segment::COMMIT_SCHEMA, + &self.snapshot.config, + )?; + let checkpoint_stream = if new_slice.checkpoint_files.is_empty() { + futures::stream::iter(files.into_iter().map(Ok)).boxed() + } else { + new_slice + .checkpoint_stream( + log_store.object_store(), + &log_segment::CHECKPOINT_SCHEMA, + &self.snapshot.config, + ) + .boxed() + }; + let mapper = LogMapper::try_new(&self.snapshot)?; + let files = ReplayStream::try_new(log_stream, checkpoint_stream, &self.snapshot)? + .map(|batch| batch.and_then(|b| mapper.map_batch(b))) + .try_collect() + .await?; + + self.files = files; + } + Ok(()) + } + + /// Get the underlying snapshot + pub(crate) fn snapshot(&self) -> &Snapshot { + &self.snapshot + } + + /// Get the table version of the snapshot + pub fn version(&self) -> i64 { + self.snapshot.version() + } + + /// Get the timestamp of the given version + pub fn version_timestamp(&self, version: i64) -> Option { + self.snapshot + .log_segment + .version_timestamp(version) + .map(|ts| ts.timestamp_millis()) + } + + /// Get the table schema of the snapshot + pub fn schema(&self) -> &StructType { + self.snapshot.schema() + } + + /// Get the table metadata of the snapshot + pub fn metadata(&self) -> &Metadata { + self.snapshot.metadata() + } + + /// Get the table protocol of the snapshot + pub fn protocol(&self) -> &Protocol { + self.snapshot.protocol() + } + + /// Get the table root of the snapshot + pub fn table_root(&self) -> Path { + self.snapshot.table_root() + } + + /// Well known table configuration + pub fn table_config(&self) -> TableConfig<'_> { + self.snapshot.table_config() + } + + /// Get a [`LogDataHandler`] for the snapshot to inspect the currently loaded state of the log. + pub fn log_data(&self) -> LogDataHandler<'_> { + LogDataHandler::new(&self.files, self.metadata(), self.schema()) + } + + /// Get the number of files in the snapshot + pub fn files_count(&self) -> usize { + self.files.iter().map(|f| f.num_rows()).sum() + } + + /// Get the files in the snapshot + pub fn file_actions(&self) -> DeltaResult + '_> { + Ok(self.files.iter().flat_map(|b| read_adds(b)).flatten()) + } + + /// Get a file action iterator for the given version + pub fn files(&self) -> impl Iterator> { + self.log_data().into_iter() + } + + /// Advance the snapshot based on the given commit actions + pub fn advance<'a>( + &mut self, + commits: impl IntoIterator, + ) -> DeltaResult { + let mut metadata = None; + let mut protocol = None; + let mut send = Vec::new(); + for commit in commits { + if metadata.is_none() { + metadata = commit.actions.iter().find_map(|a| match a { + Action::Metadata(metadata) => Some(metadata.clone()), + _ => None, + }); + } + if protocol.is_none() { + protocol = commit.actions.iter().find_map(|a| match a { + Action::Protocol(protocol) => Some(protocol.clone()), + _ => None, + }); + } + send.push(commit); + } + let actions = self.snapshot.log_segment.advance( + send, + &self.table_root(), + &log_segment::COMMIT_SCHEMA, + &self.snapshot.config, + )?; + + let mut files = Vec::new(); + let mut scanner = LogReplayScanner::new(); + + for batch in actions { + files.push(scanner.process_files_batch(&batch?, true)?); + } + + let mapper = LogMapper::try_new(&self.snapshot)?; + self.files = files + .into_iter() + .chain( + self.files + .iter() + .flat_map(|batch| scanner.process_files_batch(batch, false)), + ) + .map(|b| mapper.map_batch(b)) + .collect::>>()?; + + if let Some(metadata) = metadata { + self.snapshot.metadata = metadata; + self.snapshot.schema = serde_json::from_str(&self.snapshot.metadata.schema_string)?; + } + if let Some(protocol) = protocol { + self.snapshot.protocol = protocol; + } + + Ok(self.snapshot.version()) + } +} + +fn to_count_field(field: &StructField) -> Option { + match field.data_type() { + DataType::Map(_) | DataType::Array(_) | &DataType::BINARY => None, + DataType::Struct(s) => Some(StructField::new( + field.name(), + StructType::new( + s.fields() + .iter() + .filter_map(to_count_field) + .collect::>(), + ), + true, + )), + _ => Some(StructField::new(field.name(), DataType::LONG, true)), + } +} + +#[cfg(feature = "datafusion")] +mod datafusion { + use datafusion_common::stats::Statistics; + + use super::*; + + impl EagerSnapshot { + /// Provide table level statistics to Datafusion + pub fn datafusion_table_statistics(&self) -> Option { + self.log_data().statistics() + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use chrono::Utc; + use deltalake_test::utils::*; + use futures::TryStreamExt; + use itertools::Itertools; + + use super::log_segment::tests::test_log_segment; + use super::replay::tests::test_log_replay; + use super::*; + use crate::kernel::Remove; + use crate::protocol::{DeltaOperation, SaveMode}; + + #[tokio::test] + async fn test_snapshots() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + context.load_table(TestTables::Checkpoints).await?; + context.load_table(TestTables::Simple).await?; + context.load_table(TestTables::SimpleWithCheckpoint).await?; + context.load_table(TestTables::WithDvSmall).await?; + + test_log_segment(&context).await?; + test_log_replay(&context).await?; + test_snapshot(&context).await?; + test_eager_snapshot(&context).await?; + + Ok(()) + } + + async fn test_snapshot(context: &IntegrationContext) -> TestResult { + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + + let snapshot = + Snapshot::try_new(&Path::default(), store.clone(), Default::default(), None).await?; + + let bytes = serde_json::to_vec(&snapshot).unwrap(); + let actual: Snapshot = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual, snapshot); + + let schema_string = r#"{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,"metadata":{}}]}"#; + let expected: StructType = serde_json::from_str(schema_string)?; + assert_eq!(snapshot.schema(), &expected); + + let infos = snapshot + .commit_infos(store.clone(), None) + .await? + .try_collect::>() + .await?; + let infos = infos.into_iter().flatten().collect_vec(); + assert_eq!(infos.len(), 5); + + let tombstones = snapshot + .tombstones(store.clone())? + .try_collect::>() + .await?; + let tombstones = tombstones.into_iter().flatten().collect_vec(); + assert_eq!(tombstones.len(), 31); + + let batches = snapshot + .files(store.clone())? + .try_collect::>() + .await?; + let expected = [ + "+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+", + "| add |", + "+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+", + "| {path: part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet, partitionValues: {}, size: 262, modificationTime: 1587968626000, dataChange: true, stats: , tags: , deletionVector: , baseRowId: , defaultRowCommitVersion: , clusteringProvider: , stats_parsed: {numRecords: , minValues: , maxValues: , nullCount: }} |", + "| {path: part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet, partitionValues: {}, size: 262, modificationTime: 1587968602000, dataChange: true, stats: , tags: , deletionVector: , baseRowId: , defaultRowCommitVersion: , clusteringProvider: , stats_parsed: {numRecords: , minValues: , maxValues: , nullCount: }} |", + "| {path: part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet, partitionValues: {}, size: 429, modificationTime: 1587968602000, dataChange: true, stats: , tags: , deletionVector: , baseRowId: , defaultRowCommitVersion: , clusteringProvider: , stats_parsed: {numRecords: , minValues: , maxValues: , nullCount: }} |", + "| {path: part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet, partitionValues: {}, size: 429, modificationTime: 1587968602000, dataChange: true, stats: , tags: , deletionVector: , baseRowId: , defaultRowCommitVersion: , clusteringProvider: , stats_parsed: {numRecords: , minValues: , maxValues: , nullCount: }} |", + "| {path: part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet, partitionValues: {}, size: 429, modificationTime: 1587968602000, dataChange: true, stats: , tags: , deletionVector: , baseRowId: , defaultRowCommitVersion: , clusteringProvider: , stats_parsed: {numRecords: , minValues: , maxValues: , nullCount: }} |", + "+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+", + ]; + assert_batches_sorted_eq!(expected, &batches); + + let store = context + .table_builder(TestTables::Checkpoints) + .build_storage()? + .object_store(); + + for version in 0..=12 { + let snapshot = Snapshot::try_new( + &Path::default(), + store.clone(), + Default::default(), + Some(version), + ) + .await?; + let batches = snapshot + .files(store.clone())? + .try_collect::>() + .await?; + let num_files = batches.iter().map(|b| b.num_rows() as i64).sum::(); + assert_eq!(num_files, version); + } + + Ok(()) + } + + async fn test_eager_snapshot(context: &IntegrationContext) -> TestResult { + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + + let snapshot = + EagerSnapshot::try_new(&Path::default(), store.clone(), Default::default(), None) + .await?; + + let bytes = serde_json::to_vec(&snapshot).unwrap(); + let actual: EagerSnapshot = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual, snapshot); + + let schema_string = r#"{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,"metadata":{}}]}"#; + let expected: StructType = serde_json::from_str(schema_string)?; + assert_eq!(snapshot.schema(), &expected); + + let store = context + .table_builder(TestTables::Checkpoints) + .build_storage()? + .object_store(); + + for version in 0..=12 { + let snapshot = EagerSnapshot::try_new( + &Path::default(), + store.clone(), + Default::default(), + Some(version), + ) + .await?; + let batches = snapshot.file_actions()?.collect::>(); + assert_eq!(batches.len(), version as usize); + } + + Ok(()) + } + + #[tokio::test] + async fn test_eager_snapshot_advance() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + context.load_table(TestTables::Simple).await?; + + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + + let mut snapshot = + EagerSnapshot::try_new(&Path::default(), store.clone(), Default::default(), None) + .await?; + + let version = snapshot.version(); + + let files = snapshot.file_actions()?.enumerate().collect_vec(); + let num_files = files.len(); + + let split = files.split(|(idx, _)| *idx == num_files / 2).collect_vec(); + assert!(split.len() == 2 && !split[0].is_empty() && !split[1].is_empty()); + let (first, second) = split.into_iter().next_tuple().unwrap(); + + let removes = first + .iter() + .map(|(_, add)| { + Remove { + path: add.path.clone(), + size: Some(add.size), + data_change: add.data_change, + deletion_timestamp: Some(Utc::now().timestamp_millis()), + extended_file_metadata: Some(true), + partition_values: Some(add.partition_values.clone()), + tags: add.tags.clone(), + deletion_vector: add.deletion_vector.clone(), + base_row_id: add.base_row_id, + default_row_commit_version: add.default_row_commit_version, + } + .into() + }) + .collect_vec(); + + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + + let actions = vec![CommitData::new(removes, operation, HashMap::new()).unwrap()]; + + let new_version = snapshot.advance(&actions)?; + assert_eq!(new_version, version + 1); + + let new_files = snapshot.file_actions()?.map(|f| f.path).collect::>(); + assert_eq!(new_files.len(), num_files - first.len()); + assert!(first + .iter() + .all(|(_, add)| { !new_files.contains(&add.path) })); + assert!(second + .iter() + .all(|(_, add)| { new_files.contains(&add.path) })); + + Ok(()) + } +} diff --git a/crates/core/src/kernel/snapshot/parse.rs b/crates/core/src/kernel/snapshot/parse.rs new file mode 100644 index 0000000000..0070880c9b --- /dev/null +++ b/crates/core/src/kernel/snapshot/parse.rs @@ -0,0 +1,236 @@ +//! Utilities for converting Arrow arrays into Delta data structures. + +use arrow_array::{ + Array, BooleanArray, Int32Array, Int64Array, ListArray, MapArray, StringArray, StructArray, +}; +use percent_encoding::percent_decode_str; + +use crate::kernel::arrow::extract::{self as ex, ProvidesColumnByName}; +use crate::kernel::{Add, DeletionVectorDescriptor, Metadata, Protocol, Remove}; +use crate::{DeltaResult, DeltaTableError}; + +pub(super) fn read_metadata(batch: &dyn ProvidesColumnByName) -> DeltaResult> { + if let Some(arr) = ex::extract_and_cast_opt::(batch, "metaData") { + let id = ex::extract_and_cast::(arr, "id")?; + let name = ex::extract_and_cast::(arr, "name")?; + let description = ex::extract_and_cast::(arr, "description")?; + // let format = ex::extract_and_cast::(arr, "format")?; + let schema_string = ex::extract_and_cast::(arr, "schemaString")?; + let partition_columns = ex::extract_and_cast_opt::(arr, "partitionColumns"); + let configuration = ex::extract_and_cast_opt::(arr, "configuration"); + let created_time = ex::extract_and_cast::(arr, "createdTime")?; + + for idx in 0..arr.len() { + if arr.is_valid(idx) { + return Ok(Some(Metadata { + id: ex::read_str(id, idx)?.to_string(), + name: ex::read_str_opt(name, idx).map(|s| s.to_string()), + description: ex::read_str_opt(description, idx).map(|s| s.to_string()), + format: Default::default(), + schema_string: ex::read_str(schema_string, idx)?.to_string(), + partition_columns: collect_string_list(&partition_columns, idx) + .unwrap_or_default(), + configuration: configuration + .and_then(|pv| collect_map(&pv.value(idx)).map(|m| m.collect())) + .unwrap_or_default(), + created_time: ex::read_primitive_opt(created_time, idx), + })); + } + } + } + Ok(None) +} + +pub(super) fn read_protocol(batch: &dyn ProvidesColumnByName) -> DeltaResult> { + if let Some(arr) = ex::extract_and_cast_opt::(batch, "protocol") { + let min_reader_version = ex::extract_and_cast::(arr, "minReaderVersion")?; + let min_writer_version = ex::extract_and_cast::(arr, "minWriterVersion")?; + let maybe_reader_features = ex::extract_and_cast_opt::(arr, "readerFeatures"); + let maybe_writer_features = ex::extract_and_cast_opt::(arr, "writerFeatures"); + + for idx in 0..arr.len() { + if arr.is_valid(idx) { + return Ok(Some(Protocol { + min_reader_version: ex::read_primitive(min_reader_version, idx)?, + min_writer_version: ex::read_primitive(min_writer_version, idx)?, + reader_features: collect_string_list(&maybe_reader_features, idx) + .map(|v| v.into_iter().map(Into::into).collect()), + writer_features: collect_string_list(&maybe_writer_features, idx) + .map(|v| v.into_iter().map(Into::into).collect()), + })); + } + } + } + Ok(None) +} + +pub(super) fn read_adds(array: &dyn ProvidesColumnByName) -> DeltaResult> { + let mut result = Vec::new(); + + if let Some(arr) = ex::extract_and_cast_opt::(array, "add") { + let path = ex::extract_and_cast::(arr, "path")?; + let pvs = ex::extract_and_cast_opt::(arr, "partitionValues"); + let size = ex::extract_and_cast::(arr, "size")?; + let modification_time = ex::extract_and_cast::(arr, "modificationTime")?; + let data_change = ex::extract_and_cast::(arr, "dataChange")?; + let stats = ex::extract_and_cast::(arr, "stats")?; + let tags = ex::extract_and_cast_opt::(arr, "tags"); + let dv = ex::extract_and_cast_opt::(arr, "deletionVector"); + + let get_dv: Box Option> = if let Some(d) = dv { + let storage_type = ex::extract_and_cast::(d, "storageType")?; + let path_or_inline_dv = ex::extract_and_cast::(d, "pathOrInlineDv")?; + let offset = ex::extract_and_cast::(d, "offset")?; + let size_in_bytes = ex::extract_and_cast::(d, "sizeInBytes")?; + let cardinality = ex::extract_and_cast::(d, "cardinality")?; + + Box::new(|idx: usize| { + if ex::read_str(storage_type, idx).is_ok() { + Some(DeletionVectorDescriptor { + storage_type: std::str::FromStr::from_str( + ex::read_str(storage_type, idx).ok()?, + ) + .ok()?, + path_or_inline_dv: ex::read_str(path_or_inline_dv, idx).ok()?.to_string(), + offset: ex::read_primitive_opt(offset, idx), + size_in_bytes: ex::read_primitive(size_in_bytes, idx).ok()?, + cardinality: ex::read_primitive(cardinality, idx).ok()?, + }) + } else { + None + } + }) + } else { + Box::new(|_| None) + }; + + for i in 0..arr.len() { + if arr.is_valid(i) { + let path_ = ex::read_str(path, i)?; + let path_ = percent_decode_str(path_) + .decode_utf8() + .map_err(|_| DeltaTableError::Generic("illegal path encoding".into()))? + .to_string(); + result.push(Add { + path: path_, + size: ex::read_primitive(size, i)?, + modification_time: ex::read_primitive(modification_time, i)?, + data_change: ex::read_bool(data_change, i)?, + stats: ex::read_str_opt(stats, i).map(|s| s.to_string()), + partition_values: pvs + .and_then(|pv| collect_map(&pv.value(i)).map(|m| m.collect())) + .unwrap_or_default(), + tags: tags.and_then(|t| collect_map(&t.value(i)).map(|m| m.collect())), + deletion_vector: get_dv(i), + base_row_id: None, + default_row_commit_version: None, + clustering_provider: None, + stats_parsed: None, + }); + } + } + } + + Ok(result) +} + +pub(super) fn read_removes(array: &dyn ProvidesColumnByName) -> DeltaResult> { + let mut result = Vec::new(); + + if let Some(arr) = ex::extract_and_cast_opt::(array, "remove") { + let path = ex::extract_and_cast::(arr, "path")?; + let data_change = ex::extract_and_cast::(arr, "dataChange")?; + let deletion_timestamp = ex::extract_and_cast::(arr, "deletionTimestamp")?; + + let extended_file_metadata = + ex::extract_and_cast_opt::(arr, "extendedFileMetadata"); + let pvs = ex::extract_and_cast_opt::(arr, "partitionValues"); + let size = ex::extract_and_cast_opt::(arr, "size"); + let tags = ex::extract_and_cast_opt::(arr, "tags"); + let dv = ex::extract_and_cast_opt::(arr, "deletionVector"); + + let get_dv: Box Option> = if let Some(d) = dv { + let storage_type = ex::extract_and_cast::(d, "storageType")?; + let path_or_inline_dv = ex::extract_and_cast::(d, "pathOrInlineDv")?; + let offset = ex::extract_and_cast::(d, "offset")?; + let size_in_bytes = ex::extract_and_cast::(d, "sizeInBytes")?; + let cardinality = ex::extract_and_cast::(d, "cardinality")?; + + Box::new(|idx: usize| { + if ex::read_str(storage_type, idx).is_ok() { + Some(DeletionVectorDescriptor { + storage_type: std::str::FromStr::from_str( + ex::read_str(storage_type, idx).ok()?, + ) + .ok()?, + path_or_inline_dv: ex::read_str(path_or_inline_dv, idx).ok()?.to_string(), + offset: ex::read_primitive_opt(offset, idx), + size_in_bytes: ex::read_primitive(size_in_bytes, idx).ok()?, + cardinality: ex::read_primitive(cardinality, idx).ok()?, + }) + } else { + None + } + }) + } else { + Box::new(|_| None) + }; + + for i in 0..arr.len() { + if arr.is_valid(i) { + let path_ = ex::read_str(path, i)?; + let path_ = percent_decode_str(path_) + .decode_utf8() + .map_err(|_| DeltaTableError::Generic("illegal path encoding".into()))? + .to_string(); + result.push(Remove { + path: path_, + data_change: ex::read_bool(data_change, i)?, + deletion_timestamp: ex::read_primitive_opt(deletion_timestamp, i), + extended_file_metadata: extended_file_metadata + .and_then(|e| ex::read_bool_opt(e, i)), + size: size.and_then(|s| ex::read_primitive_opt(s, i)), + partition_values: pvs + .and_then(|pv| collect_map(&pv.value(i)).map(|m| m.collect())), + tags: tags.and_then(|t| collect_map(&t.value(i)).map(|m| m.collect())), + deletion_vector: get_dv(i), + base_row_id: None, + default_row_commit_version: None, + }); + } + } + } + + Ok(result) +} + +fn collect_map(val: &StructArray) -> Option)> + '_> { + let keys = val + .column(0) + .as_ref() + .as_any() + .downcast_ref::()?; + let values = val + .column(1) + .as_ref() + .as_any() + .downcast_ref::()?; + Some( + keys.iter() + .zip(values.iter()) + .filter_map(|(k, v)| k.map(|kv| (kv.to_string(), v.map(|vv| vv.to_string())))), + ) +} + +fn collect_string_list(arr: &Option<&ListArray>, idx: usize) -> Option> { + arr.and_then(|val| { + let values = val.value(idx); + let values = values.as_ref().as_any().downcast_ref::()?; + Some( + values + .iter() + .filter_map(|v| v.map(|vv| vv.to_string())) + .collect(), + ) + }) +} diff --git a/crates/core/src/kernel/snapshot/replay.rs b/crates/core/src/kernel/snapshot/replay.rs new file mode 100644 index 0000000000..71408b27d5 --- /dev/null +++ b/crates/core/src/kernel/snapshot/replay.rs @@ -0,0 +1,396 @@ +use std::pin::Pin; +use std::sync::Arc; +use std::task::Context; +use std::task::Poll; + +use arrow_arith::boolean::{is_not_null, or}; +use arrow_array::{ + Array, ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray, StructArray, +}; +use arrow_schema::{ + DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, + SchemaRef as ArrowSchemaRef, +}; +use arrow_select::filter::filter_record_batch; +use futures::Stream; +use hashbrown::HashSet; +use itertools::Itertools; +use percent_encoding::percent_decode_str; +use pin_project_lite::pin_project; +use tracing::debug; + +use crate::kernel::arrow::extract::{self as ex, ProvidesColumnByName}; +use crate::kernel::arrow::json; +use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; + +use super::Snapshot; + +pin_project! { + pub struct ReplayStream { + scanner: LogReplayScanner, + + mapper: Arc, + + #[pin] + commits: S, + + #[pin] + checkpoint: S, + } +} + +impl ReplayStream { + pub(super) fn try_new(commits: S, checkpoint: S, snapshot: &Snapshot) -> DeltaResult { + let stats_schema = Arc::new((&snapshot.stats_schema()?).try_into()?); + let mapper = Arc::new(LogMapper { + stats_schema, + config: snapshot.config.clone(), + }); + Ok(Self { + commits, + checkpoint, + mapper, + scanner: LogReplayScanner::new(), + }) + } +} + +pub(super) struct LogMapper { + stats_schema: ArrowSchemaRef, + config: DeltaTableConfig, +} + +impl LogMapper { + pub(super) fn try_new(snapshot: &Snapshot) -> DeltaResult { + Ok(Self { + stats_schema: Arc::new((&snapshot.stats_schema()?).try_into()?), + config: snapshot.config.clone(), + }) + } + + pub fn map_batch(&self, batch: RecordBatch) -> DeltaResult { + map_batch(batch, self.stats_schema.clone(), &self.config) + } +} + +fn map_batch( + batch: RecordBatch, + stats_schema: ArrowSchemaRef, + config: &DeltaTableConfig, +) -> DeltaResult { + let stats_col = ex::extract_and_cast_opt::(&batch, "add.stats"); + let stats_parsed_col = ex::extract_and_cast_opt::(&batch, "add.stats_parsed"); + if stats_parsed_col.is_some() { + return Ok(batch); + } + if let Some(stats) = stats_col { + let stats: Arc = + Arc::new(json::parse_json(stats, stats_schema.clone(), config)?.into()); + let schema = batch.schema(); + let add_col = ex::extract_and_cast::(&batch, "add")?; + let (add_idx, _) = schema.column_with_name("add").unwrap(); + let add_type = add_col + .fields() + .iter() + .cloned() + .chain(std::iter::once(Arc::new(ArrowField::new( + "stats_parsed", + ArrowDataType::Struct(stats_schema.fields().clone()), + true, + )))) + .collect_vec(); + let new_add = Arc::new(StructArray::try_new( + add_type.clone().into(), + add_col + .columns() + .iter() + .cloned() + .chain(std::iter::once(stats as ArrayRef)) + .collect(), + add_col.nulls().cloned(), + )?); + let new_add_field = Arc::new(ArrowField::new( + "add", + ArrowDataType::Struct(add_type.into()), + true, + )); + let mut fields = schema.fields().to_vec(); + let _ = std::mem::replace(&mut fields[add_idx], new_add_field); + let mut columns = batch.columns().to_vec(); + let _ = std::mem::replace(&mut columns[add_idx], new_add); + return Ok(RecordBatch::try_new( + Arc::new(ArrowSchema::new(fields)), + columns, + )?); + } + + Ok(batch) +} + +impl Stream for ReplayStream +where + S: Stream>, +{ + type Item = DeltaResult; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.project(); + let res = this.commits.poll_next(cx).map(|b| match b { + Some(Ok(batch)) => match this.scanner.process_files_batch(&batch, true) { + Ok(filtered) => Some(this.mapper.map_batch(filtered)), + Err(e) => Some(Err(e)), + }, + Some(Err(e)) => Some(Err(e)), + None => None, + }); + if matches!(res, Poll::Ready(None)) { + this.checkpoint.poll_next(cx).map(|b| match b { + Some(Ok(batch)) => match this.scanner.process_files_batch(&batch, false) { + Ok(filtered) => Some(this.mapper.map_batch(filtered)), + Err(e) => Some(Err(e)), + }, + Some(Err(e)) => Some(Err(e)), + None => None, + }) + } else { + res + } + } + + fn size_hint(&self) -> (usize, Option) { + let (l_com, u_com) = self.commits.size_hint(); + let (l_cp, u_cp) = self.checkpoint.size_hint(); + ( + l_com + l_cp, + u_com.and_then(|u_com| u_cp.map(|u_cp| u_com + u_cp)), + ) + } +} + +#[derive(Debug)] +pub(super) struct FileInfo<'a> { + pub path: &'a str, + pub dv: Option>, +} + +#[derive(Debug)] +pub(super) struct DVInfo<'a> { + pub storage_type: &'a str, + pub path_or_inline_dv: &'a str, + pub offset: Option, + // pub size_in_bytes: i32, + // pub cardinality: i64, +} + +fn seen_key(info: &FileInfo<'_>) -> String { + let path = percent_decode_str(info.path).decode_utf8_lossy(); + if let Some(dv) = &info.dv { + if let Some(offset) = &dv.offset { + format!( + "{}::{}{}@{offset}", + path, dv.storage_type, dv.path_or_inline_dv + ) + } else { + format!("{}::{}{}", path, dv.storage_type, dv.path_or_inline_dv) + } + } else { + path.to_string() + } +} + +pub(super) struct LogReplayScanner { + // filter: Option, + /// A set of (data file path, dv_unique_id) pairs that have been seen thus + /// far in the log. This is used to filter out files with Remove actions as + /// well as duplicate entries in the log. + seen: HashSet, +} + +impl LogReplayScanner { + /// Creates a new [`LogReplayScanner`] instance. + pub fn new() -> Self { + Self { + seen: HashSet::new(), + } + } + + /// Takes a record batch of add and protentially remove actions and returns a + /// filtered batch of actions that contains only active rows. + pub(super) fn process_files_batch( + &mut self, + batch: &RecordBatch, + is_log_batch: bool, + ) -> DeltaResult { + let add_col = ex::extract_and_cast::(batch, "add")?; + let maybe_remove_col = ex::extract_and_cast_opt::(batch, "remove"); + let filter = if let Some(remove_col) = maybe_remove_col { + or(&is_not_null(add_col)?, &is_not_null(remove_col)?)? + } else { + is_not_null(add_col)? + }; + + let filtered = filter_record_batch(batch, &filter)?; + let add_col = ex::extract_and_cast::(&filtered, "add")?; + let maybe_remove_col = ex::extract_and_cast_opt::(&filtered, "remove"); + let add_actions = read_file_info(add_col)?; + + let mut keep = Vec::with_capacity(filtered.num_rows()); + if let Some(remove_col) = maybe_remove_col { + let remove_actions = read_file_info(remove_col)?; + for (a, r) in add_actions.into_iter().zip(remove_actions.into_iter()) { + match (a, r) { + (Some(a), None) => { + let file_id = seen_key(&a); + if !self.seen.contains(&file_id) { + is_log_batch.then(|| self.seen.insert(file_id)); + keep.push(true); + } else { + keep.push(false); + } + } + (None, Some(r)) => { + self.seen.insert(seen_key(&r)); + keep.push(false); + } + // NOTE: there sould always be only one action per row. + (None, None) => debug!("WARNING: no action found for row"), + (Some(a), Some(r)) => { + debug!( + "WARNING: both add and remove actions found for row: {:?} {:?}", + a, r + ) + } + } + } + } else { + for a in add_actions.into_iter().flatten() { + let file_id = seen_key(&a); + if !self.seen.contains(&file_id) { + is_log_batch.then(|| self.seen.insert(file_id)); + keep.push(true); + } else { + keep.push(false); + } + } + }; + + let projection = filtered + .schema() + .fields() + .iter() + .enumerate() + .filter_map(|(idx, field)| (field.name() == "add").then_some(idx)) + .collect::>(); + let filtered = filtered.project(&projection)?; + + Ok(filter_record_batch(&filtered, &BooleanArray::from(keep))?) + } +} + +fn read_file_info<'a>(arr: &'a dyn ProvidesColumnByName) -> DeltaResult>>> { + let path = ex::extract_and_cast::(arr, "path")?; + let dv = ex::extract_and_cast_opt::(arr, "deletionVector"); + + let get_dv: Box DeltaResult>>> = if let Some(d) = dv { + let storage_type = ex::extract_and_cast::(d, "storageType")?; + let path_or_inline_dv = ex::extract_and_cast::(d, "pathOrInlineDv")?; + let offset = ex::extract_and_cast::(d, "offset")?; + + Box::new(|idx: usize| { + if ex::read_str(storage_type, idx).is_ok() { + Ok(Some(DVInfo { + storage_type: ex::read_str(storage_type, idx)?, + path_or_inline_dv: ex::read_str(path_or_inline_dv, idx)?, + offset: ex::read_primitive_opt(offset, idx), + })) + } else { + Ok(None) + } + }) + } else { + Box::new(|_| Ok(None)) + }; + + let mut adds = Vec::with_capacity(path.len()); + for idx in 0..path.len() { + let value = path + .is_valid(idx) + .then(|| { + Ok::<_, DeltaTableError>(FileInfo { + path: ex::read_str(path, idx)?, + dv: get_dv(idx)?, + }) + }) + .transpose()?; + adds.push(value); + } + Ok(adds) +} + +#[cfg(test)] +pub(super) mod tests { + use std::sync::Arc; + + use arrow_select::concat::concat_batches; + use deltalake_test::utils::*; + use futures::TryStreamExt; + use object_store::path::Path; + + use super::super::log_segment::LogSegment; + use super::*; + use crate::kernel::{models::ActionType, StructType}; + + pub(crate) async fn test_log_replay(context: &IntegrationContext) -> TestResult { + let log_schema = Arc::new(StructType::new(vec![ + ActionType::Add.schema_field().clone(), + ActionType::Remove.schema_field().clone(), + ])); + + let store = context + .table_builder(TestTables::SimpleWithCheckpoint) + .build_storage()? + .object_store(); + + let segment = LogSegment::try_new(&Path::default(), Some(9), store.as_ref()).await?; + let mut scanner = LogReplayScanner::new(); + + let batches = segment + .commit_stream(store.clone(), &log_schema, &Default::default())? + .try_collect::>() + .await?; + let batch = concat_batches(&batches[0].schema(), &batches)?; + assert_eq!(batch.schema().fields().len(), 2); + let filtered = scanner.process_files_batch(&batch, true)?; + assert_eq!(filtered.schema().fields().len(), 1); + + // TODO enable once we do selection pushdown in parquet read + // assert_eq!(batch.schema().fields().len(), 1); + let filtered = scanner.process_files_batch(&batch, true)?; + assert_eq!(filtered.schema().fields().len(), 1); + + let store = context + .table_builder(TestTables::Simple) + .build_storage()? + .object_store(); + let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let batches = segment + .commit_stream(store.clone(), &log_schema, &Default::default())? + .try_collect::>() + .await?; + + let batch = concat_batches(&batches[0].schema(), &batches)?; + let arr_add = batch.column_by_name("add").unwrap(); + let add_count = arr_add.len() - arr_add.null_count(); + let arr_rm = batch.column_by_name("remove").unwrap(); + let rm_count = arr_rm.len() - arr_rm.null_count(); + + let filtered = scanner.process_files_batch(&batch, true)?; + let arr_add = filtered.column_by_name("add").unwrap(); + let add_count_after = arr_add.len() - arr_add.null_count(); + assert_eq!(arr_add.null_count(), 0); + assert!(add_count_after < add_count); + assert_eq!(add_count_after, add_count - rm_count); + + Ok(()) + } +} diff --git a/crates/core/src/kernel/snapshot/serde.rs b/crates/core/src/kernel/snapshot/serde.rs new file mode 100644 index 0000000000..5162c4a1fe --- /dev/null +++ b/crates/core/src/kernel/snapshot/serde.rs @@ -0,0 +1,185 @@ +use arrow_ipc::reader::FileReader; +use arrow_ipc::writer::FileWriter; +use chrono::{TimeZone, Utc}; +use object_store::ObjectMeta; +use serde::de::{self, Deserializer, SeqAccess, Visitor}; +use serde::{ser::SerializeSeq, Deserialize, Serialize}; +use std::fmt; + +use super::log_segment::LogSegment; +use super::EagerSnapshot; + +#[derive(Serialize, Deserialize, Debug)] +struct FileInfo { + path: String, + size: usize, + last_modified: i64, + e_tag: Option, + version: Option, +} + +impl Serialize for LogSegment { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let commit_files = self + .commit_files + .iter() + .map(|f| FileInfo { + path: f.location.to_string(), + size: f.size, + last_modified: f.last_modified.timestamp_nanos_opt().unwrap(), + e_tag: f.e_tag.clone(), + version: f.version.clone(), + }) + .collect::>(); + let checkpoint_files = self + .checkpoint_files + .iter() + .map(|f| FileInfo { + path: f.location.to_string(), + size: f.size, + last_modified: f.last_modified.timestamp_nanos_opt().unwrap(), + e_tag: f.e_tag.clone(), + version: f.version.clone(), + }) + .collect::>(); + + let mut seq = serializer.serialize_seq(None)?; + seq.serialize_element(&self.version)?; + seq.serialize_element(&commit_files)?; + seq.serialize_element(&checkpoint_files)?; + seq.end() + } +} + +// Deserialize the log segment +struct LogSegmentVisitor; + +impl<'de> Visitor<'de> for LogSegmentVisitor { + type Value = LogSegment; + + fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + formatter.write_str("struct LogSegment") + } + + fn visit_seq(self, mut seq: V) -> Result + where + V: SeqAccess<'de>, + { + let version = seq + .next_element()? + .ok_or_else(|| de::Error::invalid_length(0, &self))?; + let commit_files: Vec = seq + .next_element()? + .ok_or_else(|| de::Error::invalid_length(1, &self))?; + let checkpoint_files: Vec = seq + .next_element()? + .ok_or_else(|| de::Error::invalid_length(2, &self))?; + + Ok(LogSegment { + version, + commit_files: commit_files + .into_iter() + .map(|f| { + let seconds = f.last_modified / 1_000_000_000; + let nano_seconds = (f.last_modified % 1_000_000_000) as u32; + ObjectMeta { + location: f.path.into(), + size: f.size, + last_modified: Utc.timestamp_opt(seconds, nano_seconds).single().unwrap(), + version: f.version, + e_tag: f.e_tag, + } + }) + .collect(), + checkpoint_files: checkpoint_files + .into_iter() + .map(|f| ObjectMeta { + location: f.path.into(), + size: f.size, + last_modified: Utc.from_utc_datetime( + &chrono::NaiveDateTime::from_timestamp_millis(f.last_modified).unwrap(), + ), + version: None, + e_tag: None, + }) + .collect(), + }) + } +} + +impl<'de> Deserialize<'de> for LogSegment { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + deserializer.deserialize_seq(LogSegmentVisitor) + } +} + +impl Serialize for EagerSnapshot { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut seq = serializer.serialize_seq(None)?; + seq.serialize_element(&self.snapshot)?; + for batch in self.files.iter() { + let mut buffer = vec![]; + let mut writer = FileWriter::try_new(&mut buffer, batch.schema().as_ref()) + .map_err(serde::ser::Error::custom)?; + writer.write(batch).map_err(serde::ser::Error::custom)?; + writer.finish().map_err(serde::ser::Error::custom)?; + let data = writer.into_inner().map_err(serde::ser::Error::custom)?; + seq.serialize_element(&data)?; + } + seq.end() + } +} + +// Deserialize the eager snapshot +struct EagerSnapshotVisitor; + +impl<'de> Visitor<'de> for EagerSnapshotVisitor { + type Value = EagerSnapshot; + + fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + formatter.write_str("struct EagerSnapshot") + } + + fn visit_seq(self, mut seq: V) -> Result + where + V: SeqAccess<'de>, + { + println!("eager: {:?}", "start"); + let snapshot = seq + .next_element()? + .ok_or_else(|| de::Error::invalid_length(0, &self))?; + let mut files = Vec::new(); + while let Some(elem) = seq.next_element::>()? { + let mut reader = + FileReader::try_new(std::io::Cursor::new(elem), None).map_err(|e| { + de::Error::custom(format!("failed to read ipc record batch: {}", e)) + })?; + let rb = reader + .next() + .ok_or(de::Error::custom("missing ipc data"))? + .map_err(|e| { + de::Error::custom(format!("failed to read ipc record batch: {}", e)) + })?; + files.push(rb); + } + Ok(EagerSnapshot { snapshot, files }) + } +} + +impl<'de> Deserialize<'de> for EagerSnapshot { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + deserializer.deserialize_seq(EagerSnapshotVisitor) + } +} diff --git a/crates/deltalake-core/src/lib.rs b/crates/core/src/lib.rs similarity index 65% rename from crates/deltalake-core/src/lib.rs rename to crates/core/src/lib.rs index 4f0bd64ef0..4ef9fc06fd 100644 --- a/crates/deltalake-core/src/lib.rs +++ b/crates/core/src/lib.rs @@ -6,8 +6,8 @@ //! //! ```rust //! async { -//! let table = deltalake_core::open_table("./tests/data/simple_table").await.unwrap(); -//! let files = table.get_files(); +//! let table = deltalake_core::open_table("../test/tests/data/simple_table").await.unwrap(); +//! let version = table.version(); //! }; //! ``` //! @@ -15,7 +15,7 @@ //! //! ```rust //! async { -//! let table = deltalake_core::open_table_with_version("./tests/data/simple_table", 0).await.unwrap(); +//! let table = deltalake_core::open_table_with_version("../test/tests/data/simple_table", 0).await.unwrap(); //! let files = table.get_files_by_partitions(&[deltalake_core::PartitionFilter { //! key: "month".to_string(), //! value: deltalake_core::PartitionValue::Equal("12".to_string()), @@ -28,10 +28,10 @@ //! ```rust //! async { //! let table = deltalake_core::open_table_with_ds( -//! "./tests/data/simple_table", +//! "../test/tests/data/simple_table", //! "2020-05-02T23:47:31-07:00", //! ).await.unwrap(); -//! let files = table.get_files(); +//! let version = table.version(); //! }; //! ``` //! @@ -43,19 +43,17 @@ //! - `datafusion` - enable the `datafusion::datasource::TableProvider` trait implementation //! for Delta Tables, allowing them to be queried using [DataFusion](https://github.com/apache/arrow-datafusion). //! - `datafusion-ext` - DEPRECATED: alias for `datafusion` feature. -//! - `parquet2` - use parquet2 for checkpoint deserialization. Since `arrow` and `parquet` features -//! are enabled by default for backwards compatibility, this feature needs to be used with `--no-default-features`. //! //! # Querying Delta Tables with Datafusion //! //! Querying from local filesystem: -//! ```ignore +//! ``` //! use std::sync::Arc; //! use datafusion::prelude::SessionContext; //! //! async { //! let mut ctx = SessionContext::new(); -//! let table = deltalake_core::open_table("./tests/data/simple_table") +//! let table = deltalake_core::open_table("../test/tests/data/simple_table") //! .await //! .unwrap(); //! ctx.register_table("demo", Arc::new(table)).unwrap(); @@ -67,26 +65,10 @@ //! }; //! ``` -#![deny(warnings)] #![deny(missing_docs)] #![allow(rustdoc::invalid_html_tags)] #![allow(clippy::nonminimal_bool)] -#[cfg(all(feature = "parquet", feature = "parquet2"))] -compile_error!( - "Features parquet and parquet2 are mutually exclusive and cannot be enabled together" -); - -#[cfg(all(feature = "s3", feature = "s3-native-tls"))] -compile_error!( - "Features s3 and s3-native-tls are mutually exclusive and cannot be enabled together" -); - -#[cfg(all(feature = "glue", feature = "glue-native-tls"))] -compile_error!( - "Features glue and glue-native-tls are mutually exclusive and cannot be enabled together" -); - pub mod data_catalog; pub mod errors; pub mod kernel; @@ -99,7 +81,6 @@ pub mod table; #[cfg(feature = "datafusion")] pub mod delta_datafusion; -#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; use std::collections::HashMap; @@ -117,22 +98,12 @@ pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, Object pub use operations::DeltaOps; // convenience exports for consumers to avoid aligning crate versions -#[cfg(feature = "arrow")] pub use arrow; #[cfg(feature = "datafusion")] pub use datafusion; -#[cfg(feature = "parquet")] pub use parquet; -#[cfg(feature = "parquet2")] -pub use parquet2; -#[cfg(all(feature = "arrow", feature = "parquet"))] pub use protocol::checkpoints; -// needed only for integration tests -// TODO can / should we move this into the test crate? -#[cfg(feature = "integration_test")] -pub mod test_utils; - /// Creates and loads a DeltaTable from the given path with current metadata. /// Infers the storage backend to use from the scheme in the given table path. /// @@ -203,25 +174,33 @@ mod tests { #[tokio::test] async fn read_delta_2_0_table_without_version() { - let table = crate::open_table("./tests/data/delta-0.2.0").await.unwrap(); + let table = crate::open_table("../test/tests/data/delta-0.2.0") + .await + .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet"), Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), Path::from("part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet"), ] ); - let tombstones = table.get_state().all_tombstones(); + let tombstones = table + .snapshot() + .unwrap() + .all_tombstones(table.object_store().clone()) + .await + .unwrap() + .collect_vec(); assert_eq!(tombstones.len(), 4); assert!(tombstones.contains(&crate::kernel::Remove { path: "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1564524298213), data_change: false, - extended_file_metadata: Some(false), + extended_file_metadata: None, deletion_vector: None, partition_values: None, tags: None, @@ -233,7 +212,7 @@ mod tests { #[tokio::test] async fn read_delta_table_with_update() { - let path = "./tests/data/simple_table_with_checkpoint/"; + let path = "../test/tests/data/simple_table_with_checkpoint/"; let table_newest_version = crate::open_table(path).await.unwrap(); let mut table_to_update = crate::open_table_with_version(path, 0).await.unwrap(); // calling update several times should not produce any duplicates @@ -242,48 +221,48 @@ mod tests { table_to_update.update().await.unwrap(); assert_eq!( - table_newest_version.get_files_iter().collect_vec(), - table_to_update.get_files_iter().collect_vec() + table_newest_version.get_files_iter().unwrap().collect_vec(), + table_to_update.get_files_iter().unwrap().collect_vec() ); } #[tokio::test] async fn read_delta_2_0_table_with_version() { - let mut table = crate::open_table_with_version("./tests/data/delta-0.2.0", 0) + let mut table = crate::open_table_with_version("../test/tests/data/delta-0.2.0", 0) .await .unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet"), Path::from("part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet"), ], ); - table = crate::open_table_with_version("./tests/data/delta-0.2.0", 2) + table = crate::open_table_with_version("../test/tests/data/delta-0.2.0", 2) .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), Path::from("part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet"), ] ); - table = crate::open_table_with_version("./tests/data/delta-0.2.0", 3) + table = crate::open_table_with_version("../test/tests/data/delta-0.2.0", 3) .await .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet"), Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), @@ -294,35 +273,45 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_without_version() { - let table = crate::open_table("./tests/data/delta-0.8.0").await.unwrap(); + let table = crate::open_table("../test/tests/data/delta-0.8.0") + .await + .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ + Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet"), Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), - Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet") ] ); - assert_eq!(table.get_stats().count(), 2); + assert_eq!(table.get_files_count(), 2); - assert_eq!( - table - .get_stats() - .map(|x| x.unwrap().unwrap().num_records) - .sum::(), - 4 - ); + let stats = table.snapshot().unwrap().add_actions_table(true).unwrap(); - assert_eq!( - table - .get_stats() - .map(|x| x.unwrap().unwrap().null_count["value"].as_value().unwrap()) - .collect::>(), - vec![0, 0] - ); - let tombstones = table.get_state().all_tombstones(); + let num_records = stats.column_by_name("num_records").unwrap(); + let num_records = num_records + .as_any() + .downcast_ref::() + .unwrap(); + let total_records = num_records.values().iter().sum::(); + assert_eq!(total_records, 4); + + let null_counts = stats.column_by_name("null_count.value").unwrap(); + let null_counts = null_counts + .as_any() + .downcast_ref::() + .unwrap(); + null_counts.values().iter().for_each(|x| assert_eq!(*x, 0)); + + let tombstones = table + .snapshot() + .unwrap() + .all_tombstones(table.object_store().clone()) + .await + .unwrap() + .collect_vec(); assert_eq!(tombstones.len(), 1); assert!(tombstones.contains(&crate::kernel::Remove { path: "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet".to_string(), @@ -334,29 +323,31 @@ mod tests { base_row_id: None, default_row_commit_version: None, deletion_vector: None, - tags: None, + tags: Some(HashMap::new()), })); } #[tokio::test] async fn read_delta_8_0_table_with_load_version() { - let mut table = crate::open_table("./tests/data/delta-0.8.0").await.unwrap(); + let mut table = crate::open_table("../test/tests/data/delta-0.8.0") + .await + .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ - Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet"), + Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), ] ); table.load_version(0).await.unwrap(); assert_eq!(table.version(), 0); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); + assert_eq!(table.protocol().unwrap().min_writer_version, 2); + assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), Path::from("part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"), @@ -366,8 +357,7 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_with_partitions() { - let current_dir = Path::from_filesystem_path(std::env::current_dir().unwrap()).unwrap(); - let table = crate::open_table("./tests/data/delta-0.8.0-partitioned") + let table = crate::open_table("../test/tests/data/delta-0.8.0-partitioned") .await .unwrap(); @@ -383,74 +373,64 @@ mod tests { ]; assert_eq!( - table.get_files_by_partitions(&filters).unwrap(), - vec![ - Path::from("year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet"), - Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet") - ] - ); - - #[cfg(unix)] - assert_eq!( - table.get_file_uris_by_partitions(&filters).unwrap(), - vec![ - format!("/{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet", current_dir.as_ref()), - format!("/{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet", current_dir.as_ref()) - ] - ); - #[cfg(windows)] + table.get_files_by_partitions(&filters).unwrap(), + vec![ + Path::from("year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet"), + Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet") + ] + ); assert_eq!( - table.get_file_uris_by_partitions(&filters).unwrap(), - vec![ - format!("{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet", current_dir.as_ref()), - format!("{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet", current_dir.as_ref()) - ] - ); + table.get_file_uris_by_partitions(&filters).unwrap().into_iter().map(|p| std::fs::canonicalize(p).unwrap()).collect::>(), + vec![ + std::fs::canonicalize("../test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet").unwrap(), + std::fs::canonicalize("../test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet").unwrap(), + ] + ); let filters = vec![crate::PartitionFilter { key: "month".to_string(), value: crate::PartitionValue::NotEqual("2".to_string()), }]; assert_eq!( - table.get_files_by_partitions(&filters).unwrap(), - vec![ - Path::from("year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet"), - Path::from("year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet"), - Path::from("year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet"), - Path::from("year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet") - ] - ); + table.get_files_by_partitions(&filters).unwrap(), + vec![ + Path::from("year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet"), + Path::from("year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet") + ] + ); let filters = vec![crate::PartitionFilter { key: "month".to_string(), value: crate::PartitionValue::In(vec!["2".to_string(), "12".to_string()]), }]; assert_eq!( - table.get_files_by_partitions(&filters).unwrap(), - vec![ - Path::from("year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet"), - Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet"), - Path::from("year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet"), - Path::from("year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet") - ] - ); + table.get_files_by_partitions(&filters).unwrap(), + vec![ + Path::from("year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet"), + Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet") + ] + ); let filters = vec![crate::PartitionFilter { key: "month".to_string(), value: crate::PartitionValue::NotIn(vec!["2".to_string(), "12".to_string()]), }]; assert_eq!( - table.get_files_by_partitions(&filters).unwrap(), - vec![ - Path::from("year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet"), - Path::from("year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet") - ] - ); + table.get_files_by_partitions(&filters).unwrap(), + vec![ + Path::from("year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet"), + Path::from("year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet") + ] + ); } #[tokio::test] async fn read_delta_8_0_table_with_null_partition() { - let table = crate::open_table("./tests/data/delta-0.8.0-null-partition") + let table = crate::open_table("../test/tests/data/delta-0.8.0-null-partition") .await .unwrap(); @@ -479,12 +459,12 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_with_special_partition() { - let table = crate::open_table("./tests/data/delta-0.8.0-special-partition") + let table = crate::open_table("../test/tests/data/delta-0.8.0-special-partition") .await .unwrap(); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![ Path::parse( "x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" @@ -512,7 +492,7 @@ mod tests { #[tokio::test] async fn read_delta_8_0_table_partition_with_compare_op() { - let table = crate::open_table("./tests/data/delta-0.8.0-numeric-partition") + let table = crate::open_table("../test/tests/data/delta-0.8.0-numeric-partition") .await .unwrap(); @@ -539,46 +519,12 @@ mod tests { ); } - // TODO: enable this for parquet2 - #[cfg(feature = "parquet")] - #[tokio::test] - async fn read_delta_1_2_1_struct_stats_table() { - let table_uri = "./tests/data/delta-1.2.1-only-struct-stats"; - let table_from_struct_stats = crate::open_table(table_uri).await.unwrap(); - let table_from_json_stats = crate::open_table_with_version(table_uri, 1).await.unwrap(); - - fn get_stats_for_file( - table: &crate::DeltaTable, - file_name: &str, - ) -> crate::protocol::Stats { - table - .get_file_uris() - .zip(table.get_stats()) - .filter_map(|(file_uri, file_stats)| { - if file_uri.ends_with(file_name) { - file_stats.unwrap() - } else { - None - } - }) - .next() - .unwrap() - } - - let file_to_compare = "part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet"; - - assert_eq!( - get_stats_for_file(&table_from_struct_stats, file_to_compare), - get_stats_for_file(&table_from_json_stats, file_to_compare), - ); - } - #[tokio::test] async fn test_table_history() { - let path = "./tests/data/simple_table_with_checkpoint"; - let mut latest_table = crate::open_table(path).await.unwrap(); + let path = "../test/tests/data/simple_table_with_checkpoint"; + let latest_table = crate::open_table(path).await.unwrap(); - let mut table = crate::open_table_with_version(path, 1).await.unwrap(); + let table = crate::open_table_with_version(path, 1).await.unwrap(); let history1 = table.history(None).await.expect("Cannot get table history"); let history2 = latest_table @@ -597,14 +543,14 @@ mod tests { #[tokio::test] async fn test_poll_table_commits() { - let path = "./tests/data/simple_table_with_checkpoint"; + let path = "../test/tests/data/simple_table_with_checkpoint"; let mut table = crate::open_table_with_version(path, 9).await.unwrap(); let peek = table.peek_next_commit(table.version()).await.unwrap(); assert!(matches!(peek, PeekCommit::New(..))); if let PeekCommit::New(version, actions) = peek { assert_eq!(table.version(), 9); - assert!(!table.get_files_iter().any(|f| f + assert!(!table.get_files_iter().unwrap().any(|f| f == Path::from( "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet" ))); @@ -615,7 +561,7 @@ mod tests { table.update_incremental(None).await.unwrap(); assert_eq!(table.version(), 10); - assert!(table.get_files_iter().any(|f| f + assert!(table.get_files_iter().unwrap().any(|f| f == Path::from( "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet" ))); @@ -627,15 +573,15 @@ mod tests { #[tokio::test] async fn test_read_vacuumed_log() { - let path = "./tests/data/checkpoints_vacuumed"; + let path = "../test/tests/data/checkpoints_vacuumed"; let table = crate::open_table(path).await.unwrap(); assert_eq!(table.version(), 12); } #[tokio::test] async fn test_read_vacuumed_log_history() { - let path = "./tests/data/checkpoints_vacuumed"; - let mut table = crate::open_table(path).await.unwrap(); + let path = "../test/tests/data/checkpoints_vacuumed"; + let table = crate::open_table(path).await.unwrap(); // load history for table version with available log file let history = table @@ -679,12 +625,12 @@ mod tests { #[tokio::test] async fn read_delta_table_with_cdc() { - let table = crate::open_table("./tests/data/simple_table_with_cdc") + let table = crate::open_table("../test/tests/data/simple_table_with_cdc") .await .unwrap(); assert_eq!(table.version(), 2); assert_eq!( - table.get_files_iter().collect_vec(), + table.get_files_iter().unwrap().collect_vec(), vec![Path::from( "part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet" ),] @@ -693,10 +639,10 @@ mod tests { #[tokio::test()] async fn test_version_zero_table_load() { - let path = "./tests/data/COVID-19_NYT"; - let mut latest_table: DeltaTable = crate::open_table(path).await.unwrap(); + let path = "../test/tests/data/COVID-19_NYT"; + let latest_table: DeltaTable = crate::open_table(path).await.unwrap(); - let mut version_0_table = crate::open_table_with_version(path, 0).await.unwrap(); + let version_0_table = crate::open_table_with_version(path, 0).await.unwrap(); let version_0_history = version_0_table .history(None) @@ -714,7 +660,7 @@ mod tests { async fn test_fail_fast_on_not_existing_path() { use std::path::Path as FolderPath; - let non_existing_path_str = "./tests/data/folder_doesnt_exist"; + let non_existing_path_str = "../test/tests/data/folder_doesnt_exist"; // Check that there is no such path at the beginning let path_doesnt_exist = !FolderPath::new(non_existing_path_str).exists(); @@ -730,4 +676,13 @@ mod tests { DeltaTableError::InvalidTableLocation(_expected_error_msg), )) } + + /// + #[tokio::test] + async fn test_identity_column() { + let path = "../test/tests/data/issue-2152"; + let _ = crate::open_table(path) + .await + .expect("Failed to load the table"); + } } diff --git a/crates/deltalake-core/src/logstore/default_logstore.rs b/crates/core/src/logstore/default_logstore.rs similarity index 87% rename from crates/deltalake-core/src/logstore/default_logstore.rs rename to crates/core/src/logstore/default_logstore.rs index 275732fb1a..ed463e9947 100644 --- a/crates/deltalake-core/src/logstore/default_logstore.rs +++ b/crates/core/src/logstore/default_logstore.rs @@ -29,6 +29,10 @@ impl DefaultLogStore { #[async_trait::async_trait] impl LogStore for DefaultLogStore { + fn name(&self) -> String { + "DefaultLogStore".into() + } + async fn read_commit_entry(&self, version: i64) -> DeltaResult> { super::read_commit_entry(self.storage.as_ref(), version).await } @@ -54,15 +58,6 @@ impl LogStore for DefaultLogStore { self.storage.clone() } - fn to_uri(&self, location: &Path) -> String { - super::to_uri(&self.config.location, location) - } - - #[cfg(feature = "datafusion")] - fn object_store_url(&self) -> datafusion::execution::object_store::ObjectStoreUrl { - super::object_store_url(&self.config.location) - } - fn config(&self) -> &LogStoreConfig { &self.config } diff --git a/crates/deltalake-core/src/logstore/mod.rs b/crates/core/src/logstore/mod.rs similarity index 68% rename from crates/deltalake-core/src/logstore/mod.rs rename to crates/core/src/logstore/mod.rs index dd13b6bdc5..e6b4c6e2d4 100644 --- a/crates/deltalake-core/src/logstore/mod.rs +++ b/crates/core/src/logstore/mod.rs @@ -1,4 +1,5 @@ //! Delta log store. +use dashmap::DashMap; use futures::StreamExt; use lazy_static::lazy_static; use regex::Regex; @@ -8,6 +9,7 @@ use serde::{ Deserialize, Serialize, }; use std::io::{BufRead, BufReader, Cursor}; +use std::sync::OnceLock; use std::{cmp::max, collections::HashMap, sync::Arc}; use url::Url; @@ -16,19 +18,71 @@ use crate::{ kernel::Action, operations::transaction::TransactionError, protocol::{get_last_checkpoint, ProtocolError}, - storage::{commit_uri_from_version, config::StorageOptions}, + storage::{commit_uri_from_version, ObjectStoreRef, StorageOptions}, DeltaTableError, }; use bytes::Bytes; -use log::debug; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; +use tracing::{debug, warn}; #[cfg(feature = "datafusion")] use datafusion::datasource::object_store::ObjectStoreUrl; -pub mod default_logstore; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -pub mod s3; +pub(crate) mod default_logstore; + +/// Trait for generating [LogStore] implementations +pub trait LogStoreFactory: Send + Sync { + /// Create a new [LogStore] + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + Ok(default_logstore(store, location, options)) + } +} + +/// Return the [DefaultLogStore] implementation with the provided configuration options +pub fn default_logstore( + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, +) -> Arc { + Arc::new(default_logstore::DefaultLogStore::new( + store, + LogStoreConfig { + location: location.clone(), + options: options.clone(), + }, + )) +} + +#[derive(Clone, Debug, Default)] +struct DefaultLogStoreFactory {} +impl LogStoreFactory for DefaultLogStoreFactory {} + +/// Registry of [LogStoreFactory] instances +pub type FactoryRegistry = Arc>>; + +/// TODO +pub fn logstores() -> FactoryRegistry { + static REGISTRY: OnceLock = OnceLock::new(); + REGISTRY + .get_or_init(|| { + let registry = FactoryRegistry::default(); + registry.insert( + Url::parse("memory://").unwrap(), + Arc::new(DefaultLogStoreFactory::default()), + ); + registry.insert( + Url::parse("file://").unwrap(), + Arc::new(DefaultLogStoreFactory::default()), + ); + registry + }) + .clone() +} /// Sharable reference to [`LogStore`] pub type LogStoreRef = Arc; @@ -37,6 +91,56 @@ lazy_static! { static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); } +/// Return the [LogStoreRef] for the provided [Url] location +/// +/// This will use the built-in process global [crate::storage::ObjectStoreRegistry] by default +/// +/// ```rust +/// # use deltalake_core::logstore::*; +/// # use std::collections::HashMap; +/// # use url::Url; +/// let location = Url::parse("memory:///").expect("Failed to make location"); +/// let logstore = logstore_for(location, HashMap::new()).expect("Failed to get a logstore"); +/// ``` +pub fn logstore_for( + location: Url, + options: impl Into + Clone, +) -> DeltaResult { + // turn location into scheme + let scheme = Url::parse(&format!("{}://", location.scheme())) + .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; + + if let Some(entry) = crate::storage::factories().get(&scheme) { + debug!("Found a storage provider for {scheme} ({location})"); + let (store, _prefix) = entry + .value() + .parse_url_opts(&location, &options.clone().into())?; + return logstore_with(store, location, options); + } + Err(DeltaTableError::InvalidTableLocation(location.into())) +} + +/// Return the [LogStoreRef] using the given [ObjectStoreRef] +pub fn logstore_with( + store: ObjectStoreRef, + location: Url, + options: impl Into + Clone, +) -> DeltaResult { + let scheme = Url::parse(&format!("{}://", location.scheme())) + .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; + + if let Some(factory) = logstores().get(&scheme) { + debug!("Found a logstore provider for {scheme}"); + return factory.with_options(store, &location, &options.into()); + } else { + println!("Could not find a logstore for the scheme {scheme}"); + warn!("Could not find a logstore for the scheme {scheme}"); + } + Err(DeltaTableError::InvalidTableLocation( + location.clone().into(), + )) +} + /// Configuration parameters for a log store #[derive(Debug, Clone)] pub struct LogStoreConfig { @@ -58,6 +162,14 @@ pub struct LogStoreConfig { /// become visible immediately. #[async_trait::async_trait] pub trait LogStore: Sync + Send { + /// Return the name of this LogStore implementation + fn name(&self) -> String; + + /// Trigger sync operation on log store to. + async fn refresh(&self) -> DeltaResult<()> { + Ok(()) + } + /// Read data for commit entry with the given version. async fn read_commit_entry(&self, version: i64) -> DeltaResult>; @@ -78,7 +190,10 @@ pub trait LogStore: Sync + Send { fn object_store(&self) -> Arc; /// [Path] to Delta log - fn to_uri(&self, location: &Path) -> String; + fn to_uri(&self, location: &Path) -> String { + let root = &self.config().location; + to_uri(root, location) + } /// Get fully qualified uri for table root fn root_uri(&self) -> String { @@ -94,7 +209,7 @@ pub trait LogStore: Sync + Send { async fn is_delta_table_location(&self) -> DeltaResult { // TODO We should really be using HEAD here, but this fails in windows tests let object_store = self.object_store(); - let mut stream = object_store.list(Some(self.log_path())).await?; + let mut stream = object_store.list(Some(self.log_path())); if let Some(res) = stream.next().await { match res { Ok(_) => Ok(true), @@ -112,12 +227,56 @@ pub trait LogStore: Sync + Send { /// registering/fetching. In our case the scheme is hard-coded to "delta-rs", so to get a unique /// host we convert the location from this `LogStore` to a valid name, combining the /// original scheme, host and path with invalid characters replaced. - fn object_store_url(&self) -> ObjectStoreUrl; + fn object_store_url(&self) -> ObjectStoreUrl { + crate::logstore::object_store_url(&self.config().location) + } /// Get configuration representing configured log store. fn config(&self) -> &LogStoreConfig; } +#[cfg(feature = "datafusion")] +fn object_store_url(location: &Url) -> ObjectStoreUrl { + use object_store::path::DELIMITER; + ObjectStoreUrl::parse(format!( + "delta-rs://{}-{}{}", + location.scheme(), + location.host_str().unwrap_or("-"), + location.path().replace(DELIMITER, "-").replace(':', "-") + )) + .expect("Invalid object store url.") +} + +/// TODO +pub fn to_uri(root: &Url, location: &Path) -> String { + match root.scheme() { + "file" => { + #[cfg(windows)] + let uri = format!( + "{}/{}", + root.as_ref().trim_end_matches('/'), + location.as_ref() + ) + .replace("file:///", ""); + #[cfg(unix)] + let uri = format!( + "{}/{}", + root.as_ref().trim_end_matches('/'), + location.as_ref() + ) + .replace("file://", ""); + uri + } + _ => { + if location.as_ref().is_empty() || location.as_ref() == "/" { + root.as_ref().to_string() + } else { + format!("{}/{}", root.as_ref(), location.as_ref()) + } + } + } +} + /// Reads a commit and gets list of actions pub async fn get_actions( version: i64, @@ -199,50 +358,6 @@ lazy_static! { static ref DELTA_LOG_REGEX: Regex = Regex::new(r"(\d{20})\.(json|checkpoint).*$").unwrap(); } -fn to_uri(root: &Url, location: &Path) -> String { - match root.scheme() { - "file" => { - #[cfg(windows)] - let uri = format!( - "{}/{}", - root.as_ref().trim_end_matches('/'), - location.as_ref() - ) - .replace("file:///", ""); - #[cfg(unix)] - let uri = format!( - "{}/{}", - root.as_ref().trim_end_matches('/'), - location.as_ref() - ) - .replace("file://", ""); - uri - } - _ => { - if location.as_ref().is_empty() || location.as_ref() == "/" { - root.as_ref().to_string() - } else { - format!("{}/{}", root.as_ref(), location.as_ref()) - } - } - } -} - -#[cfg(feature = "datafusion")] -fn object_store_url(location: &Url) -> ObjectStoreUrl { - // we are certain, that the URL can be parsed, since - // we make sure when we are parsing the table uri - - use object_store::path::DELIMITER; - ObjectStoreUrl::parse(format!( - "delta-rs://{}-{}{}", - location.scheme(), - location.host_str().unwrap_or("-"), - location.path().replace(DELIMITER, "-").replace(':', "-") - )) - .expect("Invalid object store url.") -} - /// Extract version from a file name in the delta log pub fn extract_version_from_filename(name: &str) -> Option { DELTA_LOG_REGEX @@ -250,7 +365,11 @@ pub fn extract_version_from_filename(name: &str) -> Option { .map(|captures| captures.get(1).unwrap().as_str().parse().unwrap()) } -async fn get_latest_version(log_store: &dyn LogStore, current_version: i64) -> DeltaResult { +/// Default implementation for retrieving the latest version +pub async fn get_latest_version( + log_store: &dyn LogStore, + current_version: i64, +) -> DeltaResult { let version_start = match get_last_checkpoint(log_store).await { Ok(last_check_point) => last_check_point.version, Err(ProtocolError::CheckpointNotFound) => { @@ -272,7 +391,7 @@ async fn get_latest_version(log_store: &dyn LogStore, current_version: i64) -> D let prefix = Some(log_store.log_path()); let offset_path = commit_uri_from_version(max_version); let object_store = log_store.object_store(); - let mut files = object_store.list_with_offset(prefix, &offset_path).await?; + let mut files = object_store.list_with_offset(prefix, &offset_path); while let Some(obj_meta) = files.next().await { let obj_meta = obj_meta?; @@ -296,7 +415,10 @@ async fn get_latest_version(log_store: &dyn LogStore, current_version: i64) -> D } /// Read delta log for a specific version -async fn read_commit_entry(storage: &dyn ObjectStore, version: i64) -> DeltaResult> { +pub async fn read_commit_entry( + storage: &dyn ObjectStore, + version: i64, +) -> DeltaResult> { let commit_uri = commit_uri_from_version(version); match storage.get(&commit_uri).await { Ok(res) => Ok(Some(res.bytes().await?)), @@ -305,7 +427,8 @@ async fn read_commit_entry(storage: &dyn ObjectStore, version: i64) -> DeltaResu } } -async fn write_commit_entry( +/// Default implementation for writing a commit entry +pub async fn write_commit_entry( storage: &dyn ObjectStore, version: i64, tmp_commit: &Path, @@ -326,9 +449,29 @@ async fn write_commit_entry( Ok(()) } -#[cfg(feature = "datafusion")] #[cfg(test)] mod tests { + use super::*; + + #[test] + fn logstore_with_invalid_url() { + let location = Url::parse("nonexistent://table").unwrap(); + let store = logstore_for(location, HashMap::default()); + assert!(store.is_err()); + } + + #[test] + fn logstore_with_memory() { + let location = Url::parse("memory://table").unwrap(); + let store = logstore_for(location, HashMap::default()); + assert!(store.is_ok()); + } +} + +#[cfg(feature = "datafusion")] +#[cfg(test)] +mod datafusion_tests { + use super::*; use url::Url; #[tokio::test] @@ -345,8 +488,8 @@ mod tests { let url_2 = Url::parse(location_2).unwrap(); assert_ne!( - super::object_store_url(&url_1).as_str(), - super::object_store_url(&url_2).as_str(), + object_store_url(&url_1).as_str(), + object_store_url(&url_2).as_str(), ); } } diff --git a/crates/core/src/operations/cast.rs b/crates/core/src/operations/cast.rs new file mode 100644 index 0000000000..33155dedd8 --- /dev/null +++ b/crates/core/src/operations/cast.rs @@ -0,0 +1,218 @@ +//! Provide common cast functionality for callers +//! +use arrow::datatypes::DataType::Dictionary; +use arrow_array::{new_null_array, Array, ArrayRef, RecordBatch, StructArray}; +use arrow_cast::{cast_with_options, CastOptions}; +use arrow_schema::{ + ArrowError, DataType, Field as ArrowField, Fields, Schema as ArrowSchema, + SchemaRef as ArrowSchemaRef, +}; +use std::sync::Arc; + +use crate::DeltaResult; + +pub(crate) fn merge_field(left: &ArrowField, right: &ArrowField) -> Result { + if let Dictionary(_, value_type) = right.data_type() { + if value_type.equals_datatype(left.data_type()) { + return Ok(left.clone()); + } + } + if let Dictionary(_, value_type) = left.data_type() { + if value_type.equals_datatype(right.data_type()) { + return Ok(right.clone()); + } + } + let mut new_field = left.clone(); + new_field.try_merge(right)?; + Ok(new_field) +} + +pub(crate) fn merge_schema( + left: ArrowSchema, + right: ArrowSchema, +) -> Result { + let mut errors = Vec::with_capacity(left.fields().len()); + let merged_fields: Result, ArrowError> = left + .fields() + .iter() + .map(|field| { + let right_field = right.field_with_name(field.name()); + if let Ok(right_field) = right_field { + let field_or_not = merge_field(field.as_ref(), right_field); + match field_or_not { + Err(e) => { + errors.push(e.to_string()); + Err(e) + } + Ok(f) => Ok(f), + } + } else { + Ok(field.as_ref().clone()) + } + }) + .collect(); + match merged_fields { + Ok(mut fields) => { + for field in right.fields() { + if !left.field_with_name(field.name()).is_ok() { + fields.push(field.as_ref().clone()); + } + } + + Ok(ArrowSchema::new(fields)) + } + Err(e) => { + errors.push(e.to_string()); + Err(ArrowError::SchemaError(errors.join("\n"))) + } + } +} + +fn cast_struct( + struct_array: &StructArray, + fields: &Fields, + cast_options: &CastOptions, + add_missing: bool, +) -> Result>, arrow_schema::ArrowError> { + fields + .iter() + .map(|field| { + let col_or_not = struct_array.column_by_name(field.name()); + match col_or_not { + None => match add_missing { + true => Ok(new_null_array(field.data_type(), struct_array.len())), + false => Err(arrow_schema::ArrowError::SchemaError(format!( + "Could not find column {0}", + field.name() + ))), + }, + Some(col) => { + if let (DataType::Struct(_), DataType::Struct(child_fields)) = + (col.data_type(), field.data_type()) + { + let child_struct = StructArray::from(col.into_data()); + let s = + cast_struct(&child_struct, child_fields, cast_options, add_missing)?; + Ok(Arc::new(StructArray::new( + child_fields.clone(), + s, + child_struct.nulls().map(ToOwned::to_owned), + )) as ArrayRef) + } else if is_cast_required(col.data_type(), field.data_type()) { + cast_with_options(col, field.data_type(), cast_options) + } else { + Ok(col.clone()) + } + } + } + }) + .collect::, _>>() +} + +fn is_cast_required(a: &DataType, b: &DataType) -> bool { + match (a, b) { + (DataType::List(a_item), DataType::List(b_item)) => { + // If list item name is not the default('item') the list must be casted + !a.equals_datatype(b) || a_item.name() != b_item.name() + } + (_, _) => !a.equals_datatype(b), + } +} + +/// Cast recordbatch to a new target_schema, by casting each column array +pub fn cast_record_batch( + batch: &RecordBatch, + target_schema: ArrowSchemaRef, + safe: bool, + add_missing: bool, +) -> DeltaResult { + let cast_options = CastOptions { + safe, + ..Default::default() + }; + + let s = StructArray::new( + batch.schema().as_ref().to_owned().fields, + batch.columns().to_owned(), + None, + ); + let columns = cast_struct(&s, target_schema.fields(), &cast_options, add_missing)?; + Ok(RecordBatch::try_new(target_schema, columns)?) +} + +#[cfg(test)] +mod tests { + use crate::operations::cast::{cast_record_batch, is_cast_required}; + use arrow::array::ArrayData; + use arrow_array::{Array, ArrayRef, ListArray, RecordBatch}; + use arrow_buffer::Buffer; + use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; + use std::sync::Arc; + + #[test] + fn test_cast_record_batch_with_list_non_default_item() { + let array = Arc::new(make_list_array()) as ArrayRef; + let source_schema = Schema::new(vec![Field::new( + "list_column", + array.data_type().clone(), + false, + )]); + let record_batch = RecordBatch::try_new(Arc::new(source_schema), vec![array]).unwrap(); + + let fields = Fields::from(vec![Field::new_list( + "list_column", + Field::new("item", DataType::Int8, false), + false, + )]); + let target_schema = Arc::new(Schema::new(fields)) as SchemaRef; + + let result = cast_record_batch(&record_batch, target_schema, false, false); + + let schema = result.unwrap().schema(); + let field = schema.column_with_name("list_column").unwrap().1; + if let DataType::List(list_item) = field.data_type() { + assert_eq!(list_item.name(), "item"); + } else { + panic!("Not a list"); + } + } + + fn make_list_array() -> ListArray { + let value_data = ArrayData::builder(DataType::Int32) + .len(8) + .add_buffer(Buffer::from_slice_ref([0, 1, 2, 3, 4, 5, 6, 7])) + .build() + .unwrap(); + + let value_offsets = Buffer::from_slice_ref([0, 3, 6, 8]); + + let list_data_type = DataType::List(Arc::new(Field::new("element", DataType::Int32, true))); + let list_data = ArrayData::builder(list_data_type) + .len(3) + .add_buffer(value_offsets) + .add_child_data(value_data) + .build() + .unwrap(); + ListArray::from(list_data) + } + + #[test] + fn test_is_cast_required_with_list() { + let field1 = DataType::List(FieldRef::from(Field::new("item", DataType::Int32, false))); + let field2 = DataType::List(FieldRef::from(Field::new("item", DataType::Int32, false))); + + assert!(!is_cast_required(&field1, &field2)); + } + + #[test] + fn test_is_cast_required_with_list_non_default_item() { + let field1 = DataType::List(FieldRef::from(Field::new("item", DataType::Int32, false))); + let field2 = DataType::List(FieldRef::from(Field::new( + "element", + DataType::Int32, + false, + ))); + + assert!(is_cast_required(&field1, &field2)); + } +} diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs new file mode 100644 index 0000000000..ee5ed16da9 --- /dev/null +++ b/crates/core/src/operations/constraints.rs @@ -0,0 +1,408 @@ +//! Add a check constraint to a table + +use std::sync::Arc; + +use datafusion::execution::context::SessionState; +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; +use datafusion_common::ToDFSchema; +use futures::future::BoxFuture; +use futures::StreamExt; + +use crate::delta_datafusion::expr::fmt_expr_to_sql; +use crate::delta_datafusion::{ + register_store, DeltaDataChecker, DeltaScanBuilder, DeltaSessionContext, +}; +use crate::kernel::{Protocol, WriterFeatures}; +use crate::logstore::LogStoreRef; +use crate::operations::datafusion_utils::Expression; +use crate::protocol::DeltaOperation; +use crate::table::state::DeltaTableState; +use crate::table::Constraint; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; + +use super::datafusion_utils::into_expr; +use super::transaction::{CommitBuilder, CommitProperties}; + +/// Build a constraint to add to a table +pub struct ConstraintBuilder { + /// A snapshot of the table's state + snapshot: DeltaTableState, + /// Name of the constraint + name: Option, + /// Constraint expression + expr: Option, + /// Delta object store for handling data files + log_store: LogStoreRef, + /// Datafusion session state relevant for executing the input plan + state: Option, + /// Additional information to add to the commit + commit_properties: CommitProperties, +} + +impl ConstraintBuilder { + /// Create a new builder + pub fn new(log_store: LogStoreRef, snapshot: DeltaTableState) -> Self { + Self { + name: None, + expr: None, + snapshot, + log_store, + state: None, + commit_properties: CommitProperties::default(), + } + } + + /// Specify the constraint to be added + pub fn with_constraint, E: Into>( + mut self, + name: S, + expression: E, + ) -> Self { + self.name = Some(name.into()); + self.expr = Some(expression.into()); + self + } + + /// Specify the datafusion session context + pub fn with_session_state(mut self, state: SessionState) -> Self { + self.state = Some(state); + self + } + + /// Additional metadata to be added to commit info + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + 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; + + Box::pin(async move { + let name = match this.name { + Some(v) => v, + None => return Err(DeltaTableError::Generic("No name provided".to_string())), + }; + + let expr = this + .expr + .ok_or_else(|| DeltaTableError::Generic("No Expresion provided".to_string()))?; + + let mut metadata = this.snapshot.metadata().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", + name + ))); + } + + let state = this.state.unwrap_or_else(|| { + let session: SessionContext = DeltaSessionContext::default().into(); + register_store(this.log_store.clone(), session.runtime_env()); + session.state() + }); + + let scan = DeltaScanBuilder::new(&this.snapshot, this.log_store.clone(), &state) + .build() + .await?; + + let schema = scan.schema().to_dfschema()?; + let expr = into_expr(expr, &schema, &state)?; + let expr_str = fmt_expr_to_sql(&expr)?; + + // Checker built here with the one time constraint to check. + let checker = + DeltaDataChecker::new_with_constraints(vec![Constraint::new("*", &expr_str)]); + + 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. + + metadata.configuration.insert( + format!("delta.constraints.{}", name), + Some(expr_str.clone()), + ); + + let old_protocol = this.snapshot.protocol(); + let protocol = Protocol { + min_reader_version: if old_protocol.min_reader_version > 1 { + old_protocol.min_reader_version + } else { + 1 + }, + min_writer_version: if old_protocol.min_writer_version > 3 { + old_protocol.min_writer_version + } else { + 3 + }, + reader_features: old_protocol.reader_features.clone(), + writer_features: if old_protocol.min_writer_version < 7 { + old_protocol.writer_features.clone() + } else { + let current_features = old_protocol.writer_features.clone(); + if let Some(mut features) = current_features { + features.insert(WriterFeatures::CheckConstraints); + Some(features) + } else { + current_features + } + }, + }; + + let operation = DeltaOperation::AddConstraint { + name: name.clone(), + expr: expr_str.clone(), + }; + + let actions = vec![metadata.into(), protocol.into()]; + + let commit = CommitBuilder::from(this.commit_properties) + .with_actions(actions) + .build(Some(&this.snapshot), this.log_store.clone(), operation)? + .await?; + + this.snapshot + .merge(commit.data.actions, &commit.data.operation, commit.version)?; + Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) + }) + } +} + +#[cfg(feature = "datafusion")] +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType as ArrowDataType, Field, Schema as ArrowSchema}; + use datafusion_expr::{col, lit}; + + use crate::writer::test_utils::{create_bare_table, get_arrow_schema, get_record_batch}; + use crate::{DeltaOps, DeltaResult, DeltaTable}; + + fn get_constraint(table: &DeltaTable, name: &str) -> String { + table + .metadata() + .unwrap() + .configuration + .get(name) + .unwrap() + .clone() + .unwrap() + } + + async fn get_constraint_op_params(table: &mut DeltaTable) -> String { + let commit_info = table.history(None).await.unwrap(); + let last_commit = &commit_info[0]; + last_commit + .operation_parameters + .as_ref() + .unwrap() + .get("expr") + .unwrap() + .as_str() + .unwrap() + .to_owned() + } + + #[tokio::test] + async fn add_constraint_with_invalid_data() -> DeltaResult<()> { + 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("id", "value > 5") + .await; + assert!(constraint.is_err()); + Ok(()) + } + + #[tokio::test] + async fn add_valid_constraint() -> DeltaResult<()> { + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + + let mut table = table + .add_constraint() + .with_constraint("id", "value < 1000") + .await?; + let version = table.version(); + assert_eq!(version, 1); + + let expected_expr = "value < 1000"; + assert_eq!(get_constraint_op_params(&mut table).await, expected_expr); + assert_eq!( + get_constraint(&table, "delta.constraints.id"), + expected_expr + ); + Ok(()) + } + + #[tokio::test] + async fn add_constraint_datafusion() -> DeltaResult<()> { + // Add constraint by providing a datafusion expression. + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + + let mut table = table + .add_constraint() + .with_constraint("valid_values", col("value").lt(lit(1000))) + .await?; + let version = table.version(); + assert_eq!(version, 1); + + let expected_expr = "value < 1000"; + assert_eq!(get_constraint_op_params(&mut table).await, expected_expr); + assert_eq!( + get_constraint(&table, "delta.constraints.valid_values"), + expected_expr + ); + + Ok(()) + } + + #[tokio::test] + async fn test_constraint_case_sensitive() -> DeltaResult<()> { + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("Id", ArrowDataType::Utf8, true), + Field::new("vAlue", ArrowDataType::Int32, true), + Field::new("mOdifieD", ArrowDataType::Utf8, true), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&arrow_schema.clone()), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["B", "C", "X"])), + Arc::new(arrow::array::Int32Array::from(vec![10, 20, 30])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2023-07-04", + "2023-07-04", + ])), + ], + ) + .unwrap(); + + let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + + let mut table = DeltaOps(table) + .add_constraint() + .with_constraint("valid_values", "vAlue < 1000") + .await?; + let version = table.version(); + assert_eq!(version, 1); + + let expected_expr = "vAlue < 1000"; + assert_eq!(get_constraint_op_params(&mut table).await, expected_expr); + assert_eq!( + get_constraint(&table, "delta.constraints.valid_values"), + expected_expr + ); + + Ok(()) + } + + #[tokio::test] + async fn add_conflicting_named_constraint() -> DeltaResult<()> { + 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", "value < 60") + .await?; + + let new_table = DeltaOps(new_table); + let second_constraint = new_table + .add_constraint() + .with_constraint("id", "value < 10") + .await; + assert!(second_constraint.is_err()); + Ok(()) + } + + #[tokio::test] + async fn write_data_that_violates_constraint() -> DeltaResult<()> { + 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; + assert!(err.is_err()); + Ok(()) + } + + #[tokio::test] + async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { + 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()); + Ok(()) + } +} diff --git a/crates/deltalake-core/src/operations/convert_to_delta.rs b/crates/core/src/operations/convert_to_delta.rs similarity index 76% rename from crates/deltalake-core/src/operations/convert_to_delta.rs rename to crates/core/src/operations/convert_to_delta.rs index 97cb08f560..351a596062 100644 --- a/crates/deltalake-core/src/operations/convert_to_delta.rs +++ b/crates/core/src/operations/convert_to_delta.rs @@ -2,21 +2,19 @@ // https://github.com/delta-io/delta/blob/1d5dd774111395b0c4dc1a69c94abc169b1c83b6/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala use crate::{ - kernel::{Action, Add, Schema, StructField}, + kernel::{Add, DataType, Schema, StructField}, logstore::{LogStore, LogStoreRef}, operations::create::CreateBuilder, protocol::SaveMode, - storage::config::configure_log_store, + table::builder::ensure_table_uri, table::config::DeltaConfigKey, - DeltaResult, DeltaTable, DeltaTableError, DeltaTablePartition, ObjectStoreError, - NULL_PARTITION_VALUE_DATA_PATH, + DeltaResult, DeltaTable, DeltaTableError, ObjectStoreError, NULL_PARTITION_VALUE_DATA_PATH, }; use arrow::{datatypes::Schema as ArrowSchema, error::ArrowError}; use futures::{ future::{self, BoxFuture}, TryStreamExt, }; -use log::{debug, info}; use parquet::{ arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}, errors::ParquetError, @@ -29,6 +27,7 @@ use std::{ str::{FromStr, Utf8Error}, sync::Arc, }; +use tracing::debug; /// Error converting a Parquet table to a Delta table #[derive(Debug, thiserror::Error)] @@ -55,6 +54,8 @@ enum Error { DeltaTableAlready, #[error("Location must be provided to convert a Parquet table to a Delta table")] MissingLocation, + #[error("The location provided must be a valid URL")] + InvalidLocation(#[from] url::ParseError), } impl From for DeltaTableError { @@ -225,13 +226,16 @@ impl ConvertToDeltaBuilder { } /// Consume self into CreateBuilder with corresponding add actions, schemas and operation meta - async fn into_create_builder(mut self) -> Result { + async fn into_create_builder(self) -> Result { // Use the specified log store. If a log store is not provided, create a new store from the specified path. // Return an error if neither log store nor path is provided let log_store = if let Some(log_store) = self.log_store { log_store } else if let Some(location) = self.location { - configure_log_store(&location, self.storage_options.unwrap_or_default(), None)? + crate::logstore::logstore_for( + ensure_table_uri(location)?, + self.storage_options.unwrap_or_default(), + )? } else { return Err(Error::MissingLocation); }; @@ -240,7 +244,7 @@ impl ConvertToDeltaBuilder { if log_store.is_delta_table_location().await? { return Err(Error::DeltaTableAlready); } - info!( + debug!( "Converting Parquet table in log store location: {:?}", log_store.root_uri() ); @@ -250,7 +254,6 @@ impl ConvertToDeltaBuilder { let mut files = Vec::new(); object_store .list(None) - .await? .try_for_each_concurrent(10, |meta| { if Some("parquet") == meta.location.extension() { debug!("Found parquet file {:#?}", meta.location); @@ -259,6 +262,7 @@ impl ConvertToDeltaBuilder { future::ready(Ok(())) }) .await?; + if files.is_empty() { return Err(Error::ParquetFileNotFound); } @@ -266,60 +270,87 @@ impl ConvertToDeltaBuilder { // Iterate over the parquet files. Parse partition columns, generate add actions and collect parquet file schemas let mut arrow_schemas = Vec::new(); let mut actions = Vec::new(); + // partition columns that were defined by caller and are expected to apply on this table + let mut expected_partitions: HashMap = self + .partition_schema + .clone() + .into_iter() + .map(|field| (field.name.clone(), field)) + .collect(); // A HashSet of all unique partition columns in a Parquet table let mut partition_columns = HashSet::new(); // A vector of StructField of all unique partition columns in a Parquet table - let mut partition_schema_fields = Vec::new(); + let mut partition_schema_fields = HashMap::new(); + for file in files { // A HashMap from partition column to value for this parquet file only let mut partition_values = HashMap::new(); - let mut iter = file.location.as_ref().split('/').peekable(); + let location = file.location.clone().to_string(); + let mut iter = location.split('/').peekable(); let mut subpath = iter.next(); + // Get partitions from subpaths. Skip the last subpath while iter.peek().is_some() { - if let Some(subpath) = subpath { - // Return an error if the partition is not hive-partitioning - let partition = DeltaTablePartition::try_from( - percent_decode_str(subpath).decode_utf8()?.as_ref(), - )?; - debug!( - "Found partition {partition:#?} in parquet file {:#?}", - file.location - ); - let (key, val) = (partition.key, partition.value); - partition_values.insert( - key.clone(), - if val == NULL_PARTITION_VALUE_DATA_PATH { - None - } else { - Some(val) - }, - ); - if partition_columns.insert(key.clone()) { - if let Some(schema) = self.partition_schema.take(key.as_str()) { - partition_schema_fields.push(schema); - } else { - // Return an error if the schema of a partition column is not provided by user - return Err(Error::MissingPartitionSchema); - } + let curr_path = subpath.unwrap(); + let (key, value) = curr_path + .split_once('=') + .ok_or(Error::MissingPartitionSchema)?; + + if partition_columns.insert(key.to_string()) { + if let Some(schema) = expected_partitions.remove(key) { + partition_schema_fields.insert(key.to_string(), schema); + } else { + // Return an error if the schema of a partition column is not provided by user + return Err(Error::MissingPartitionSchema); } + } + + // Safety: we just checked that the key is present in the map + let field = partition_schema_fields.get(key).unwrap(); + let scalar = if value == NULL_PARTITION_VALUE_DATA_PATH { + Ok(crate::kernel::Scalar::Null(field.data_type().clone())) } else { - // This error shouldn't happen. The while condition ensures that subpath is not none - panic!("Subpath iterator index overflows"); + let decoded = percent_decode_str(value).decode_utf8()?; + match field.data_type() { + DataType::Primitive(p) => p.parse_scalar(decoded.as_ref()), + _ => Err(crate::kernel::Error::Generic(format!( + "Exprected primitive type, found: {:?}", + field.data_type() + ))), + } } + .map_err(|_| Error::MissingPartitionSchema)?; + + partition_values.insert(key.to_string(), scalar); + subpath = iter.next(); } - actions.push(Action::Add(Add { - path: percent_decode_str(file.location.as_ref()) - .decode_utf8()? - .to_string(), - size: i64::try_from(file.size)?, - partition_values, - modification_time: file.last_modified.timestamp_millis(), - data_change: true, - ..Default::default() - })); + actions.push( + Add { + path: percent_decode_str(file.location.as_ref()) + .decode_utf8()? + .to_string(), + size: i64::try_from(file.size)?, + partition_values: partition_values + .into_iter() + .map(|(k, v)| { + ( + k, + if v.is_null() { + None + } else { + Some(v.serialize()) + }, + ) + }) + .collect(), + modification_time: file.last_modified.timestamp_millis(), + data_change: true, + ..Default::default() + } + .into(), + ); let mut arrow_schema = ParquetRecordBatchStreamBuilder::new(ParquetObjectReader::new( object_store.clone(), @@ -329,13 +360,14 @@ impl ConvertToDeltaBuilder { .schema() .as_ref() .clone(); + // Arrow schema of Parquet files may have conflicting metatdata // Since Arrow schema metadata is not used to generate Delta table schema, we set the metadata field to an empty HashMap arrow_schema.metadata = HashMap::new(); arrow_schemas.push(arrow_schema); } - if !self.partition_schema.is_empty() { + if !expected_partitions.is_empty() { // Partition column provided by the user does not exist in the parquet files return Err(Error::PartitionColumnNotExist(self.partition_schema)); } @@ -345,8 +377,12 @@ impl ConvertToDeltaBuilder { let mut schema_fields = Schema::try_from(&ArrowSchema::try_merge(arrow_schemas)?)? .fields() .clone(); - schema_fields.append(&mut partition_schema_fields); - debug!("Schema fields for the parquet table: {schema_fields:#?}"); + schema_fields.append( + &mut partition_schema_fields + .values() + .cloned() + .collect::>(), + ); // Generate CreateBuilder with corresponding add actions, schemas and operation meta let mut builder = CreateBuilder::new() @@ -365,6 +401,7 @@ impl ConvertToDeltaBuilder { if let Some(metadata) = self.metadata { builder = builder.with_metadata(metadata); } + Ok(builder) } } @@ -389,11 +426,11 @@ impl std::future::IntoFuture for ConvertToDeltaBuilder { #[cfg(test)] mod tests { - use super::{configure_log_store, ConvertToDeltaBuilder, DeltaTable, LogStoreRef, StructField}; + use super::*; use crate::{ - kernel::schema::{DataType, PrimitiveType}, + kernel::{DataType, PrimitiveType, Scalar}, open_table, - storage::config::StorageOptions, + storage::StorageOptions, Path, }; use itertools::Itertools; @@ -424,7 +461,9 @@ mod tests { } fn log_store(path: impl Into) -> LogStoreRef { - configure_log_store(&path.into(), StorageOptions::default(), None) + let path: String = path.into(); + let location = ensure_table_uri(path).expect("Failed to get the URI from the path"); + crate::logstore::logstore_for(location, StorageOptions::default()) .expect("Failed to create an object store") } @@ -442,7 +481,9 @@ mod tests { // Copy all files to a temp directory to perform testing. Skip Delta log copy_files(format!("{}/{}", env!("CARGO_MANIFEST_DIR"), path), temp_dir); let builder = if from_path { - ConvertToDeltaBuilder::new().with_location(temp_dir) + ConvertToDeltaBuilder::new().with_location( + ensure_table_uri(temp_dir).expect("Failed to turn temp dir into a URL"), + ) } else { ConvertToDeltaBuilder::new().with_log_store(log_store(temp_dir)) }; @@ -482,7 +523,7 @@ mod tests { expected_version: i64, expected_paths: Vec, expected_schema: Vec, - expected_partition_values: &[(String, Option)], + expected_partition_values: &[(String, Scalar)], ) { assert_eq!( table.version(), @@ -490,7 +531,7 @@ mod tests { "Testing location: {test_data_from:?}" ); - let mut files = table.get_files_iter().collect_vec(); + let mut files = table.get_files_iter().unwrap().collect_vec(); files.sort(); assert_eq!( files, expected_paths, @@ -509,28 +550,38 @@ mod tests { ); let mut partition_values = table - .get_partition_values() - .flat_map(|map| map.clone()) + .snapshot() + .unwrap() + .log_data() + .into_iter() + .flat_map(|add| { + add.partition_values() + .unwrap() + .iter() + .map(|(k, v)| (k.to_string(), v.clone())) + .collect::>() + }) .collect::>(); - partition_values.sort(); + partition_values.sort_by_key(|(k, v)| (k.clone(), v.serialize())); assert_eq!(partition_values, expected_partition_values); } // Test Parquet files in object store location #[tokio::test] async fn test_convert_to_delta() { - let path = "tests/data/delta-0.8.0-date"; + let path = "../test/tests/data/delta-0.8.0-date"; let table = create_delta_table(path, Vec::new(), false).await; let action = table .get_active_add_actions_by_partitions(&[]) .expect("Failed to get Add actions") .next() - .expect("Iterator index overflows"); + .expect("Iterator index overflows") + .unwrap(); assert_eq!( - action.path, + action.path(), "part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet" ); - assert!(action.data_change); + assert_delta_table( table, path, @@ -539,13 +590,13 @@ mod tests { "part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet", )], vec![ - schema_field("date", PrimitiveType::Date, true), + StructField::new("date", DataType::DATE, true), schema_field("dayOfYear", PrimitiveType::Integer, true), ], &[], ); - let path = "tests/data/delta-0.8.0-null-partition"; + let path = "../test/tests/data/delta-0.8.0-null-partition"; let table = create_delta_table( path, vec![schema_field("k", PrimitiveType::String, true)], @@ -561,16 +612,16 @@ mod tests { Path::from("k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet") ], vec![ - schema_field("k", PrimitiveType::String, true), - schema_field("v", PrimitiveType::Long, true), + StructField::new("k", DataType::STRING, true), + StructField::new("v", DataType::LONG, true), ], &[ - ("k".to_string(), None), - ("k".to_string(), Some("A".to_string())), + ("k".to_string(), Scalar::String("A".to_string())), + ("k".to_string(), Scalar::Null(DataType::STRING)), ], ); - let path = "tests/data/delta-0.8.0-special-partition"; + let path = "../test/tests/data/delta-0.8.0-special-partition"; let table = create_delta_table( path, vec![schema_field("x", PrimitiveType::String, true)], @@ -596,12 +647,12 @@ mod tests { schema_field("y", PrimitiveType::Long, true), ], &[ - ("x".to_string(), Some("A/A".to_string())), - ("x".to_string(), Some("B B".to_string())), + ("x".to_string(), Scalar::String("A/A".to_string())), + ("x".to_string(), Scalar::String("B B".to_string())), ], ); - let path = "tests/data/delta-0.8.0-partitioned"; + let path = "../test/tests/data/delta-0.8.0-partitioned"; let table = create_delta_table( path, vec![ @@ -643,24 +694,24 @@ mod tests { schema_field("year", PrimitiveType::String, true), ], &[ - ("day".to_string(), Some("1".to_string())), - ("day".to_string(), Some("20".to_string())), - ("day".to_string(), Some("3".to_string())), - ("day".to_string(), Some("4".to_string())), - ("day".to_string(), Some("5".to_string())), - ("day".to_string(), Some("5".to_string())), - ("month".to_string(), Some("1".to_string())), - ("month".to_string(), Some("12".to_string())), - ("month".to_string(), Some("12".to_string())), - ("month".to_string(), Some("2".to_string())), - ("month".to_string(), Some("2".to_string())), - ("month".to_string(), Some("4".to_string())), - ("year".to_string(), Some("2020".to_string())), - ("year".to_string(), Some("2020".to_string())), - ("year".to_string(), Some("2020".to_string())), - ("year".to_string(), Some("2021".to_string())), - ("year".to_string(), Some("2021".to_string())), - ("year".to_string(), Some("2021".to_string())), + ("day".to_string(), Scalar::String("1".to_string())), + ("day".to_string(), Scalar::String("20".to_string())), + ("day".to_string(), Scalar::String("3".to_string())), + ("day".to_string(), Scalar::String("4".to_string())), + ("day".to_string(), Scalar::String("5".to_string())), + ("day".to_string(), Scalar::String("5".to_string())), + ("month".to_string(), Scalar::String("1".to_string())), + ("month".to_string(), Scalar::String("12".to_string())), + ("month".to_string(), Scalar::String("12".to_string())), + ("month".to_string(), Scalar::String("2".to_string())), + ("month".to_string(), Scalar::String("2".to_string())), + ("month".to_string(), Scalar::String("4".to_string())), + ("year".to_string(), Scalar::String("2020".to_string())), + ("year".to_string(), Scalar::String("2020".to_string())), + ("year".to_string(), Scalar::String("2020".to_string())), + ("year".to_string(), Scalar::String("2021".to_string())), + ("year".to_string(), Scalar::String("2021".to_string())), + ("year".to_string(), Scalar::String("2021".to_string())), ], ); } @@ -668,7 +719,7 @@ mod tests { // Test opening the newly created Delta table #[tokio::test] async fn test_open_created_delta_table() { - let path = "tests/data/delta-0.2.0"; + let path = "../test/tests/data/delta-0.2.0"; let table = open_created_delta_table(path, Vec::new()).await; assert_delta_table( table, @@ -687,7 +738,7 @@ mod tests { &[], ); - let path = "tests/data/delta-0.8-empty"; + let path = "../test/tests/data/delta-0.8-empty"; let table = open_created_delta_table(path, Vec::new()).await; assert_delta_table( table, @@ -701,7 +752,7 @@ mod tests { &[], ); - let path = "tests/data/delta-0.8.0"; + let path = "../test/tests/data/delta-0.8.0"; let table = open_created_delta_table(path, Vec::new()).await; assert_delta_table( table, @@ -720,7 +771,7 @@ mod tests { // Test Parquet files in path #[tokio::test] async fn test_convert_to_delta_from_path() { - let path = "tests/data/delta-2.2.0-partitioned-types"; + let path = "../test/tests/data/delta-2.2.0-partitioned-types"; let table = create_delta_table( path, vec![ @@ -751,16 +802,16 @@ mod tests { schema_field("c3", PrimitiveType::Integer, true), ], &[ - ("c1".to_string(), Some("4".to_string())), - ("c1".to_string(), Some("5".to_string())), - ("c1".to_string(), Some("6".to_string())), - ("c2".to_string(), Some("a".to_string())), - ("c2".to_string(), Some("b".to_string())), - ("c2".to_string(), Some("c".to_string())), + ("c1".to_string(), Scalar::Integer(4)), + ("c1".to_string(), Scalar::Integer(5)), + ("c1".to_string(), Scalar::Integer(6)), + ("c2".to_string(), Scalar::String("a".to_string())), + ("c2".to_string(), Scalar::String("b".to_string())), + ("c2".to_string(), Scalar::String("c".to_string())), ], ); - let path = "tests/data/delta-0.8.0-numeric-partition"; + let path = "../test/tests/data/delta-0.8.0-numeric-partition"; let table = create_delta_table( path, vec![ @@ -788,10 +839,10 @@ mod tests { schema_field("z", PrimitiveType::String, true), ], &[ - ("x".to_string(), Some("10".to_string())), - ("x".to_string(), Some("9".to_string())), - ("y".to_string(), Some("10.0".to_string())), - ("y".to_string(), Some("9.9".to_string())), + ("x".to_string(), Scalar::Long(10)), + ("x".to_string(), Scalar::Long(9)), + ("y".to_string(), Scalar::Double(10.0)), + ("y".to_string(), Scalar::Double(9.9)), ], ); } @@ -819,7 +870,7 @@ mod tests { #[tokio::test] async fn test_partition_column_not_exist() { let _table = ConvertToDeltaBuilder::new() - .with_location("tests/data/delta-0.8.0-null-partition") + .with_location("../test/tests/data/delta-0.8.0-null-partition") .with_partition_schema(vec![schema_field("foo", PrimitiveType::String, true)]) .await .expect_err( @@ -830,7 +881,7 @@ mod tests { #[tokio::test] async fn test_missing_partition_schema() { let _table = ConvertToDeltaBuilder::new() - .with_location("tests/data/delta-0.8.0-numeric-partition") + .with_location("../test/tests/data/delta-0.8.0-numeric-partition") .await .expect_err("The schema of a partition column is not provided by user. Should error"); } @@ -838,7 +889,7 @@ mod tests { #[tokio::test] async fn test_delta_table_already() { let _table = ConvertToDeltaBuilder::new() - .with_location("tests/data/delta-0.2.0") + .with_location("../test/tests/data/delta-0.2.0") .await .expect_err("The given location is already a delta table location. Should error"); } diff --git a/crates/deltalake-core/src/operations/create.rs b/crates/core/src/operations/create.rs similarity index 80% rename from crates/deltalake-core/src/operations/create.rs rename to crates/core/src/operations/create.rs index 0dca038f4a..ccf1ba6bd9 100644 --- a/crates/deltalake-core/src/operations/create.rs +++ b/crates/core/src/operations/create.rs @@ -1,20 +1,21 @@ //! Command for creating a new delta table // https://github.com/delta-io/delta/blob/master/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use futures::future::BoxFuture; -use serde_json::{Map, Value}; +use serde_json::Value; -use super::transaction::{commit, PROTOCOL}; +use super::transaction::{CommitBuilder, TableReference, PROTOCOL}; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Action, DataType, Metadata, Protocol, StructField, StructType}; +use crate::kernel::{ + Action, DataType, Metadata, Protocol, ReaderFeatures, StructField, StructType, WriterFeatures, +}; use crate::logstore::{LogStore, LogStoreRef}; use crate::protocol::{DeltaOperation, SaveMode}; use crate::table::builder::ensure_table_uri; use crate::table::config::DeltaConfigKey; -use crate::table::DeltaTableMetaData; use crate::{DeltaTable, DeltaTableBuilder}; #[derive(thiserror::Error, Debug)] @@ -56,7 +57,7 @@ pub struct CreateBuilder { actions: Vec, log_store: Option, configuration: HashMap>, - metadata: Option>, + metadata: Option>, } impl Default for CreateBuilder { @@ -181,8 +182,11 @@ impl CreateBuilder { /// /// This might include provenance information such as an id of the /// user that made the commit or the program that created it. - pub fn with_metadata(mut self, metadata: Map) -> Self { - self.metadata = Some(metadata); + pub fn with_metadata( + mut self, + metadata: impl IntoIterator, + ) -> Self { + self.metadata = Some(HashMap::from_iter(metadata)); self } @@ -231,8 +235,45 @@ impl CreateBuilder { ) }; + let contains_timestampntz = &self + .columns + .iter() + .any(|f| f.data_type() == &DataType::TIMESTAMPNTZ); + // TODO configure more permissive versions based on configuration. Also how should this ideally be handled? // We set the lowest protocol we can, and if subsequent writes use newer features we update metadata? + + let (min_reader_version, min_writer_version, writer_features, reader_features) = + if *contains_timestampntz { + let mut converted_writer_features = self + .configuration + .keys() + .map(|key| key.clone().into()) + .filter(|v| !matches!(v, WriterFeatures::Other(_))) + .collect::>(); + + let mut converted_reader_features = self + .configuration + .keys() + .map(|key| key.clone().into()) + .filter(|v| !matches!(v, ReaderFeatures::Other(_))) + .collect::>(); + converted_writer_features.insert(WriterFeatures::TimestampWithoutTimezone); + converted_reader_features.insert(ReaderFeatures::TimestampWithoutTimezone); + ( + 3, + 7, + Some(converted_writer_features), + Some(converted_reader_features), + ) + } else { + ( + PROTOCOL.default_reader_version(), + PROTOCOL.default_writer_version(), + None, + None, + ) + }; let protocol = self .actions .iter() @@ -242,20 +283,24 @@ impl CreateBuilder { _ => unreachable!(), }) .unwrap_or_else(|| Protocol { - min_reader_version: PROTOCOL.default_reader_version(), - min_writer_version: PROTOCOL.default_writer_version(), - writer_features: None, - reader_features: None, + min_reader_version, + min_writer_version, + writer_features, + reader_features, }); - let metadata = DeltaTableMetaData::new( - self.name, - self.comment, - None, + let mut metadata = Metadata::try_new( StructType::new(self.columns), self.partition_columns.unwrap_or_default(), self.configuration, - ); + )? + .with_created_time(chrono::Utc::now().timestamp_millis()); + if let Some(name) = self.name { + metadata = metadata.with_name(name); + } + if let Some(comment) = self.comment { + metadata = metadata.with_description(comment); + } let operation = DeltaOperation::Create { mode: self.mode.clone(), @@ -264,10 +309,7 @@ impl CreateBuilder { protocol: protocol.clone(), }; - let mut actions = vec![ - Action::Protocol(protocol), - Action::Metadata(Metadata::try_from(metadata)?), - ]; + let mut actions = vec![Action::Protocol(protocol), Action::Metadata(metadata)]; actions.extend( self.actions .into_iter() @@ -286,8 +328,10 @@ impl std::future::IntoFuture for CreateBuilder { let this = self; Box::pin(async move { let mode = this.mode.clone(); + let app_metadata = this.metadata.clone().unwrap_or_default(); let (mut table, actions, operation) = this.into_table_and_actions()?; let log_store = table.log_store(); + let table_state = if log_store.is_delta_table_location().await? { match mode { SaveMode::ErrorIfExists => return Err(CreateError::TableAlreadyExists.into()), @@ -298,21 +342,23 @@ impl std::future::IntoFuture for CreateBuilder { } SaveMode::Overwrite => { table.load().await?; - &table.state + Some(table.snapshot()?) } } } else { - &table.state + None }; - let version = commit( - table.log_store.as_ref(), - &actions, - operation, - table_state, - None, - ) - .await?; + let version = CommitBuilder::default() + .with_actions(actions) + .with_app_metadata(app_metadata) + .build( + table_state.map(|f| f as &dyn TableReference), + table.log_store.clone(), + operation, + )? + .await? + .version(); table.load_version(version).await?; Ok(table) @@ -320,13 +366,13 @@ impl std::future::IntoFuture for CreateBuilder { } } -#[cfg(all(test, feature = "parquet"))] +#[cfg(test)] mod tests { use super::*; use crate::operations::DeltaOps; use crate::table::config::DeltaConfigKey; use crate::writer::test_utils::get_delta_schema; - use tempdir::TempDir; + use tempfile::TempDir; #[tokio::test] async fn test_create() { @@ -345,7 +391,7 @@ mod tests { #[tokio::test] async fn test_create_local_relative_path() { let table_schema = get_delta_schema(); - let tmp_dir = TempDir::new_in(".", "tmp_").unwrap(); + let tmp_dir = TempDir::new_in(".").unwrap(); let relative_path = format!( "./{}", tmp_dir.path().file_name().unwrap().to_str().unwrap() @@ -365,7 +411,7 @@ mod tests { #[tokio::test] async fn test_create_table_local_path() { let schema = get_delta_schema(); - let tmp_dir = TempDir::new_in(".", "tmp_").unwrap(); + let tmp_dir = TempDir::new_in(".").unwrap(); let relative_path = format!( "./{}", tmp_dir.path().file_name().unwrap().to_str().unwrap() @@ -388,11 +434,11 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); assert_eq!( - table.protocol().min_reader_version, + table.protocol().unwrap().min_reader_version, PROTOCOL.default_reader_version() ); assert_eq!( - table.protocol().min_writer_version, + table.protocol().unwrap().min_writer_version, PROTOCOL.default_writer_version() ); assert_eq!(table.get_schema().unwrap(), &schema); @@ -410,8 +456,8 @@ mod tests { .with_actions(vec![Action::Protocol(protocol)]) .await .unwrap(); - assert_eq!(table.protocol().min_reader_version, 0); - assert_eq!(table.protocol().min_writer_version, 0); + assert_eq!(table.protocol().unwrap().min_reader_version, 0); + assert_eq!(table.protocol().unwrap().min_writer_version, 0); let table = CreateBuilder::new() .with_location("memory://") diff --git a/crates/deltalake-core/src/operations/delete.rs b/crates/core/src/operations/delete.rs similarity index 81% rename from crates/deltalake-core/src/operations/delete.rs rename to crates/core/src/operations/delete.rs index 31486183be..e28633ae17 100644 --- a/crates/deltalake-core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -17,7 +17,7 @@ //! .await?; //! ```` -use std::collections::HashMap; +use core::panic; use std::sync::Arc; use std::time::{Instant, SystemTime, UNIX_EPOCH}; @@ -32,15 +32,15 @@ use datafusion_common::DFSchema; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; use serde::Serialize; -use serde_json::Value; use super::datafusion_utils::Expression; -use super::transaction::PROTOCOL; +use super::transaction::{CommitBuilder, CommitProperties, PROTOCOL}; use crate::delta_datafusion::expr::fmt_expr_to_sql; -use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; -use crate::errors::{DeltaResult, DeltaTableError}; +use crate::delta_datafusion::{ + find_files, register_store, DataFusionMixins, DeltaScanBuilder, DeltaSessionContext, +}; +use crate::errors::DeltaResult; use crate::kernel::{Action, Add, Remove}; -use crate::operations::transaction::commit; use crate::operations::write::write_execution_plan; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; @@ -59,8 +59,8 @@ pub struct DeleteBuilder { state: Option, /// Properties passed to underlying parquet writer for when files are rewritten writer_properties: Option, - /// Additional metadata to be added to commit - app_metadata: Option>, + /// Commit properties and configuration + commit_properties: CommitProperties, } #[derive(Default, Debug, Serialize)] @@ -90,7 +90,7 @@ impl DeleteBuilder { snapshot, log_store, state: None, - app_metadata: None, + commit_properties: CommitProperties::default(), writer_properties: None, } } @@ -107,12 +107,9 @@ impl DeleteBuilder { self } - /// Additional metadata to be added to commit info - pub fn with_metadata( - mut self, - metadata: impl IntoIterator, - ) -> Self { - self.app_metadata = Some(HashMap::from_iter(metadata)); + /// Additonal information to write to the commit + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; self } @@ -138,11 +135,7 @@ async fn excute_non_empty_expr( let input_schema = snapshot.input_schema()?; let input_dfschema: DFSchema = input_schema.clone().as_ref().clone().try_into()?; - let table_partition_cols = snapshot - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns - .clone(); + let table_partition_cols = snapshot.metadata().partition_columns.clone(); let scan = DeltaScanBuilder::new(snapshot, log_store.clone(), state) .with_files(rewrite) @@ -156,14 +149,13 @@ async fn excute_non_empty_expr( let predicate_expr = create_physical_expr( &negated_expression, &input_dfschema, - &input_schema, state.execution_props(), )?; let filter: Arc = Arc::new(FilterExec::try_new(predicate_expr, scan.clone())?); let add_actions = write_execution_plan( - snapshot, + Some(snapshot), state.clone(), filter.clone(), table_partition_cols.clone(), @@ -172,9 +164,15 @@ async fn excute_non_empty_expr( None, writer_properties, false, - false, + None, ) - .await?; + .await? + .into_iter() + .map(|a| match a { + Action::Add(a) => a, + _ => panic!("Expected Add action"), + }) + .collect::>(); let read_records = scan.parquet_scan.metrics().and_then(|m| m.output_rows()); let filter_records = filter.metrics().and_then(|m| m.output_rows()); @@ -192,14 +190,14 @@ async fn execute( snapshot: &DeltaTableState, state: SessionState, writer_properties: Option, - app_metadata: Option>, -) -> DeltaResult<((Vec, i64), DeleteMetrics)> { + mut commit_properties: CommitProperties, +) -> DeltaResult<((Vec, i64, Option), DeleteMetrics)> { let exec_start = Instant::now(); let mut metrics = DeleteMetrics::default(); let scan_start = Instant::now(); let candidates = find_files(snapshot, log_store.clone(), &state, predicate.clone()).await?; - metrics.scan_time_ms = Instant::now().duration_since(scan_start).as_micros(); + metrics.scan_time_ms = Instant::now().duration_since(scan_start).as_millis(); let predicate = predicate.unwrap_or(Expr::Literal(ScalarValue::Boolean(Some(true)))); @@ -228,7 +226,6 @@ async fn execute( .as_millis() as i64; let mut actions: Vec = add.into_iter().map(Action::Add).collect(); - let mut version = snapshot.version(); metrics.num_removed_files = remove.len(); metrics.num_added_files = actions.len(); @@ -247,24 +244,36 @@ async fn execute( })) } - metrics.execution_time_ms = Instant::now().duration_since(exec_start).as_micros(); + metrics.execution_time_ms = Instant::now().duration_since(exec_start).as_millis(); + + commit_properties + .app_metadata + .insert("readVersion".to_owned(), snapshot.version().into()); + commit_properties.app_metadata.insert( + "operationMetrics".to_owned(), + serde_json::to_value(&metrics)?, + ); // Do not make a commit when there are zero updates to the state - if !actions.is_empty() { - let operation = DeltaOperation::Delete { - predicate: Some(fmt_expr_to_sql(&predicate)?), - }; - version = commit( - log_store.as_ref(), - &actions, - operation, - snapshot, - app_metadata, - ) - .await?; + let operation = DeltaOperation::Delete { + predicate: Some(fmt_expr_to_sql(&predicate)?), + }; + if actions.is_empty() { + return Ok(((actions, snapshot.version(), None), metrics)); } - Ok(((actions, version), metrics)) + let commit = CommitBuilder::from(commit_properties) + .with_actions(actions) + .build(Some(snapshot), log_store, operation)? + .await?; + Ok(( + ( + commit.data.actions, + commit.version, + Some(commit.data.operation), + ), + metrics, + )) } impl std::future::IntoFuture for DeleteBuilder { @@ -275,12 +284,11 @@ impl std::future::IntoFuture for DeleteBuilder { let mut this = self; Box::pin(async move { - PROTOCOL.check_append_only(&this.snapshot)?; - - PROTOCOL.can_write_to(&this.snapshot)?; + PROTOCOL.check_append_only(&this.snapshot.snapshot)?; + PROTOCOL.can_write_to(&this.snapshot.snapshot)?; let state = this.state.unwrap_or_else(|| { - let session = SessionContext::new(); + let session: SessionContext = DeltaSessionContext::default().into(); // If a user provides their own their DF state then they must register the store themselves register_store(this.log_store.clone(), session.runtime_env()); @@ -298,20 +306,21 @@ impl std::future::IntoFuture for DeleteBuilder { None => None, }; - let ((actions, version), metrics) = execute( + let ((actions, version, operation), metrics) = execute( predicate, this.log_store.clone(), &this.snapshot, state, this.writer_properties, - this.app_metadata, + this.commit_properties, ) .await?; - this.snapshot - .merge(DeltaTableState::from_actions(actions, version)?, true, true); - let table = DeltaTable::new_with_state(this.log_store, this.snapshot); + if let Some(op) = &operation { + this.snapshot.merge(actions, op, version)?; + } + let table = DeltaTable::new_with_state(this.log_store, this.snapshot); Ok((table, metrics)) }) } @@ -331,6 +340,10 @@ mod tests { use arrow::array::Int32Array; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; + use arrow_array::ArrayRef; + use arrow_array::StructArray; + use arrow_buffer::NullBuffer; + use arrow_schema::Fields; use datafusion::assert_batches_sorted_eq; use datafusion::prelude::*; use serde_json::json; @@ -388,17 +401,25 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table).delete().await.unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 0); + assert_eq!(table.get_files_count(), 0); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_deleted_rows, None); assert_eq!(metrics.num_copied_rows, None); + let commit_info = table.history(None).await.unwrap(); + let last_commit = &commit_info[0]; + let _extra_info = last_commit.info.clone(); + // assert_eq!( + // extra_info["operationMetrics"], + // serde_json::to_value(&metrics).unwrap() + // ); + // rewrite is not required assert_eq!(metrics.rewrite_time_ms, 0); @@ -442,7 +463,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -466,15 +487,15 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .delete() .with_predicate(col("value").eq(lit(1))) .await .unwrap(); assert_eq!(table.version(), 3); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -483,7 +504,7 @@ mod tests { assert_eq!(metrics.num_copied_rows, Some(3)); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("value = 1")); @@ -622,7 +643,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .delete() @@ -630,7 +651,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); @@ -680,7 +701,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 3); + assert_eq!(table.get_files_count(), 3); let (table, metrics) = DeltaOps(table) .delete() @@ -692,7 +713,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); @@ -713,6 +734,58 @@ mod tests { assert_batches_sorted_eq!(&expected, &actual); } + #[tokio::test] + async fn test_delete_nested() { + use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + // Test Delete with a predicate that references struct fields + // See #2019 + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Utf8, true), + Field::new( + "props", + DataType::Struct(Fields::from(vec![Field::new("a", DataType::Utf8, true)])), + true, + ), + ])); + + let struct_array = StructArray::new( + Fields::from(vec![Field::new("a", DataType::Utf8, true)]), + vec![Arc::new(arrow::array::StringArray::from(vec![ + Some("2021-02-01"), + Some("2021-02-02"), + None, + None, + ])) as ArrayRef], + Some(NullBuffer::from_iter(vec![true, true, true, false])), + ); + + let data = vec![ + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "C", "D"])) as ArrayRef, + Arc::new(struct_array) as ArrayRef, + ]; + let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; + + let table = DeltaOps::new_in_memory().write(batches).await.unwrap(); + + let (table, _metrics) = DeltaOps(table) + .delete() + .with_predicate("props['a'] = '2021-02-02'") + .await + .unwrap(); + + let expected = [ + "+----+-----------------+", + "| id | props |", + "+----+-----------------+", + "| A | {a: 2021-02-01} |", + "| C | {a: } |", + "| D | |", + "+----+-----------------+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } + #[tokio::test] async fn test_failure_nondeterministic_query() { // Deletion requires a deterministic predicate diff --git a/crates/core/src/operations/drop_constraints.rs b/crates/core/src/operations/drop_constraints.rs new file mode 100644 index 0000000000..5a1e448812 --- /dev/null +++ b/crates/core/src/operations/drop_constraints.rs @@ -0,0 +1,180 @@ +//! Drop a constraint from a table + +use futures::future::BoxFuture; + +use super::transaction::{CommitBuilder, CommitProperties}; +use crate::kernel::Action; +use crate::logstore::LogStoreRef; +use crate::protocol::DeltaOperation; +use crate::table::state::DeltaTableState; +use crate::DeltaTable; +use crate::{DeltaResult, DeltaTableError}; + +/// Remove constraints from the table +pub struct DropConstraintBuilder { + /// A snapshot of the table's state + snapshot: DeltaTableState, + /// Name of the constraint + name: Option, + /// Raise if constraint doesn't exist + raise_if_not_exists: bool, + /// Delta object store for handling data files + log_store: LogStoreRef, + /// Additional information to add to the commit + commit_properties: CommitProperties, +} + +impl DropConstraintBuilder { + /// Create a new builder + pub fn new(log_store: LogStoreRef, snapshot: DeltaTableState) -> Self { + Self { + name: None, + raise_if_not_exists: true, + snapshot, + log_store, + commit_properties: CommitProperties::default(), + } + } + + /// Specify the constraint to be removed + pub fn with_constraint>(mut self, name: S) -> Self { + self.name = Some(name.into()); + self + } + + /// Specify if you want to raise if the constraint does not exist + pub fn with_raise_if_not_exists(mut self, raise: bool) -> Self { + self.raise_if_not_exists = raise; + self + } + + /// Additional metadata to be added to commit info + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + self + } +} + +impl std::future::IntoFuture for DropConstraintBuilder { + type Output = DeltaResult; + + type IntoFuture = BoxFuture<'static, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let mut this = self; + + Box::pin(async move { + let name = this + .name + .ok_or(DeltaTableError::Generic("No name provided".to_string()))?; + + let mut metadata = this.snapshot.metadata().clone(); + let configuration_key = format!("delta.constraints.{}", name); + + if metadata.configuration.remove(&configuration_key).is_none() { + if this.raise_if_not_exists { + return Err(DeltaTableError::Generic(format!( + "Constraint with name: {} doesn't exists", + name + ))); + } + return Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)); + } + let operation = DeltaOperation::DropConstraint { name: name.clone() }; + + let actions = vec![Action::Metadata(metadata)]; + + let commit = CommitBuilder::from(this.commit_properties) + .with_actions(actions) + .build(Some(&this.snapshot), this.log_store.clone(), operation)? + .await?; + + this.snapshot + .merge(commit.data.actions, &commit.data.operation, commit.version)?; + Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) + }) + } +} + +#[cfg(feature = "datafusion")] +#[cfg(test)] +mod tests { + use crate::writer::test_utils::{create_bare_table, get_record_batch}; + use crate::{DeltaOps, DeltaResult, DeltaTable}; + + async fn get_constraint_op_params(table: &mut DeltaTable) -> String { + let commit_info = table.history(None).await.unwrap(); + let last_commit = &commit_info[0]; + + last_commit + .operation_parameters + .as_ref() + .unwrap() + .get("name") + .unwrap() + .as_str() + .unwrap() + .to_owned() + } + + #[tokio::test] + async fn drop_valid_constraint() -> DeltaResult<()> { + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + + let table = table + .add_constraint() + .with_constraint("id", "value < 1000") + .await?; + + let mut table = DeltaOps(table) + .drop_constraints() + .with_constraint("id") + .await?; + + let expected_name = "id"; + assert_eq!(get_constraint_op_params(&mut table).await, expected_name); + assert_eq!(table.metadata().unwrap().configuration.get("id"), None); + Ok(()) + } + + #[tokio::test] + async fn drop_invalid_constraint_not_existing() -> DeltaResult<()> { + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + + let table = DeltaOps(write) + .drop_constraints() + .with_constraint("not_existing") + .await; + assert!(table.is_err()); + + Ok(()) + } + + #[tokio::test] + async fn drop_invalid_constraint_ignore() -> DeltaResult<()> { + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + + let version = write.version(); + + let table = DeltaOps(write) + .drop_constraints() + .with_constraint("not_existing") + .with_raise_if_not_exists(false) + .await?; + + let version_after = table.version(); + + assert_eq!(version, version_after); + Ok(()) + } +} diff --git a/crates/deltalake-core/src/operations/filesystem_check.rs b/crates/core/src/operations/filesystem_check.rs similarity index 81% rename from crates/deltalake-core/src/operations/filesystem_check.rs rename to crates/core/src/operations/filesystem_check.rs index b79f22b1f4..ada8f35251 100644 --- a/crates/deltalake-core/src/operations/filesystem_check.rs +++ b/crates/core/src/operations/filesystem_check.rs @@ -27,11 +27,13 @@ use url::{ParseError, Url}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Action, Add, Remove}; use crate::logstore::LogStoreRef; -use crate::operations::transaction::commit; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::DeltaTable; +use super::transaction::CommitBuilder; +use super::transaction::CommitProperties; + /// Audit the Delta Table's active files with the underlying file system. /// See this module's documentation for more information #[derive(Debug)] @@ -42,6 +44,8 @@ pub struct FileSystemCheckBuilder { log_store: LogStoreRef, /// Don't remove actions to the table log. Just determine which files can be removed dry_run: bool, + /// Commit properties and configuration + commit_properties: CommitProperties, } /// Details of the FSCK operation including which files were removed from the log @@ -78,6 +82,7 @@ impl FileSystemCheckBuilder { snapshot: state, log_store, dry_run: false, + commit_properties: CommitProperties::default(), } } @@ -87,23 +92,29 @@ impl FileSystemCheckBuilder { self } + /// Additonal information to write to the commit + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + self + } + async fn create_fsck_plan(&self) -> DeltaResult { - let mut files_relative: HashMap<&str, &Add> = - HashMap::with_capacity(self.snapshot.files().len()); + let mut files_relative: HashMap = + HashMap::with_capacity(self.snapshot.file_actions()?.len()); let log_store = self.log_store.clone(); - for active in self.snapshot.files() { + for active in self.snapshot.file_actions()? { if is_absolute_path(&active.path)? { return Err(DeltaTableError::Generic( "Filesystem check does not support absolute paths".to_string(), )); } else { - files_relative.insert(&active.path, active); + files_relative.insert(active.path.clone(), active); } } let object_store = log_store.object_store(); - let mut files = object_store.list(None).await?; + let mut files = object_store.list(None); while let Some(result) = files.next().await { let file = result?; files_relative.remove(file.location.as_ref()); @@ -126,7 +137,11 @@ impl FileSystemCheckBuilder { } impl FileSystemCheckPlan { - pub async fn execute(self, snapshot: &DeltaTableState) -> DeltaResult { + pub async fn execute( + self, + snapshot: &DeltaTableState, + mut commit_properties: CommitProperties, + ) -> DeltaResult { if self.files_to_remove.is_empty() { return Ok(FileSystemCheckMetrics { dry_run: false, @@ -154,21 +169,29 @@ impl FileSystemCheckPlan { default_row_commit_version: file.default_row_commit_version, })); } - - commit( - self.log_store.as_ref(), - &actions, - DeltaOperation::FileSystemCheck {}, - snapshot, - // TODO pass through metadata - None, - ) - .await?; - - Ok(FileSystemCheckMetrics { + let metrics = FileSystemCheckMetrics { dry_run: false, files_removed: removed_file_paths, - }) + }; + + commit_properties + .app_metadata + .insert("readVersion".to_owned(), snapshot.version().into()); + commit_properties.app_metadata.insert( + "operationMetrics".to_owned(), + serde_json::to_value(&metrics)?, + ); + + CommitBuilder::from(commit_properties) + .with_actions(actions) + .build( + Some(snapshot), + self.log_store.clone(), + DeltaOperation::FileSystemCheck {}, + )? + .await?; + + Ok(metrics) } } @@ -191,7 +214,7 @@ impl std::future::IntoFuture for FileSystemCheckBuilder { )); } - let metrics = plan.execute(&this.snapshot).await?; + let metrics = plan.execute(&this.snapshot, this.commit_properties).await?; let mut table = DeltaTable::new_with_state(this.log_store, this.snapshot); table.update().await?; Ok((table, metrics)) diff --git a/crates/deltalake-core/src/operations/load.rs b/crates/core/src/operations/load.rs similarity index 95% rename from crates/deltalake-core/src/operations/load.rs rename to crates/core/src/operations/load.rs index 610f86dee6..c89e63ebab 100644 --- a/crates/deltalake-core/src/operations/load.rs +++ b/crates/core/src/operations/load.rs @@ -7,6 +7,7 @@ use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use futures::future::BoxFuture; use super::transaction::PROTOCOL; +use crate::delta_datafusion::DataFusionMixins; use crate::errors::{DeltaResult, DeltaTableError}; use crate::logstore::LogStoreRef; use crate::table::state::DeltaTableState; @@ -47,10 +48,10 @@ impl std::future::IntoFuture for LoadBuilder { let this = self; Box::pin(async move { - PROTOCOL.can_read_from(&this.snapshot)?; + PROTOCOL.can_read_from(&this.snapshot.snapshot)?; let table = DeltaTable::new_with_state(this.log_store, this.snapshot); - let schema = table.state.arrow_schema()?; + let schema = table.snapshot()?.arrow_schema()?; let projection = this .columns .map(|cols| { @@ -88,7 +89,7 @@ mod tests { #[tokio::test] async fn test_load_local() -> TestResult { - let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") + let table = DeltaTableBuilder::from_uri("../test/tests/data/delta-0.8.0") .load() .await .unwrap(); diff --git a/crates/core/src/operations/merge/barrier.rs b/crates/core/src/operations/merge/barrier.rs new file mode 100644 index 0000000000..f1df28c4a4 --- /dev/null +++ b/crates/core/src/operations/merge/barrier.rs @@ -0,0 +1,677 @@ +//! Merge Barrier determines which files have modifications during the merge operation +//! +//! For every unique path in the input stream, a barrier is established. If any +//! single record for a file contains any delete, update, or insert operations +//! then the barrier for the file is opened and can be sent downstream. +//! To determine if a file contains zero changes, the input stream is +//! exhausted. Afterwards, records are then dropped. +//! +//! Bookkeeping is maintained to determine which files have modifications so +//! they can be removed from the delta log. + +use std::{ + collections::{HashMap, HashSet}, + pin::Pin, + sync::{Arc, Mutex}, + task::{Context, Poll}, +}; + +use arrow_array::{builder::UInt64Builder, ArrayRef, RecordBatch}; +use arrow_schema::SchemaRef; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, +}; +use datafusion_common::{DataFusionError, Result as DataFusionResult}; +use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNodeCore}; +use datafusion_physical_expr::{Distribution, PhysicalExpr}; +use futures::{Stream, StreamExt}; + +use crate::{ + delta_datafusion::get_path_column, + operations::merge::{TARGET_DELETE_COLUMN, TARGET_INSERT_COLUMN, TARGET_UPDATE_COLUMN}, + DeltaTableError, +}; + +pub(crate) type BarrierSurvivorSet = Arc>>; + +#[derive(Debug)] +/// Physical Node for the MergeBarrier +/// Batches to this node must be repartitioned on col('deleta_rs_path'). +/// Each record batch then undergoes further partitioning based on the file column to it's corresponding barrier +pub struct MergeBarrierExec { + input: Arc, + file_column: Arc, + survivors: BarrierSurvivorSet, + expr: Arc, +} + +impl MergeBarrierExec { + /// Create a new MergeBarrierExec Node + pub fn new( + input: Arc, + file_column: Arc, + expr: Arc, + ) -> Self { + MergeBarrierExec { + input, + file_column, + survivors: Arc::new(Mutex::new(HashSet::new())), + expr, + } + } + + /// Files that have modifications to them and need to removed from the delta log + pub fn survivors(&self) -> BarrierSurvivorSet { + self.survivors.clone() + } +} + +impl ExecutionPlan for MergeBarrierExec { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> arrow_schema::SchemaRef { + self.input.schema() + } + + fn output_partitioning(&self) -> datafusion_physical_expr::Partitioning { + self.input.output_partitioning() + } + + fn required_input_distribution(&self) -> Vec { + vec![Distribution::HashPartitioned(vec![self.expr.clone()]); 1] + } + + fn output_ordering(&self) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec> { + vec![self.input.clone()] + } + + fn with_new_children( + self: std::sync::Arc, + children: Vec>, + ) -> datafusion_common::Result> { + Ok(Arc::new(MergeBarrierExec::new( + children[0].clone(), + self.file_column.clone(), + self.expr.clone(), + ))) + } + + fn execute( + &self, + partition: usize, + context: std::sync::Arc, + ) -> datafusion_common::Result { + let input = self.input.execute(partition, context)?; + Ok(Box::pin(MergeBarrierStream::new( + input, + self.schema(), + self.survivors.clone(), + self.file_column.clone(), + ))) + } +} + +impl DisplayAs for MergeBarrierExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "MergeBarrier",)?; + Ok(()) + } + } + } +} + +#[derive(Debug)] +enum State { + Feed, + Drain, + Finalize, + Abort, + Done, +} + +#[derive(Debug)] +enum PartitionBarrierState { + Closed, + Open, +} + +#[derive(Debug)] +struct MergeBarrierPartition { + state: PartitionBarrierState, + buffer: Vec, + file_name: Option, +} + +impl MergeBarrierPartition { + pub fn new(file_name: Option) -> Self { + MergeBarrierPartition { + state: PartitionBarrierState::Closed, + buffer: Vec::new(), + file_name, + } + } + + pub fn feed(&mut self, batch: RecordBatch) -> DataFusionResult<()> { + match self.state { + PartitionBarrierState::Closed => { + let delete_count = get_count(&batch, TARGET_DELETE_COLUMN)?; + let update_count = get_count(&batch, TARGET_UPDATE_COLUMN)?; + let insert_count = get_count(&batch, TARGET_INSERT_COLUMN)?; + self.buffer.push(batch); + + if insert_count > 0 || update_count > 0 || delete_count > 0 { + self.state = PartitionBarrierState::Open; + } + } + PartitionBarrierState::Open => { + self.buffer.push(batch); + } + } + Ok(()) + } + + pub fn drain(&mut self) -> Option { + match self.state { + PartitionBarrierState::Closed => None, + PartitionBarrierState::Open => self.buffer.pop(), + } + } +} + +struct MergeBarrierStream { + schema: SchemaRef, + state: State, + input: SendableRecordBatchStream, + file_column: Arc, + survivors: BarrierSurvivorSet, + map: HashMap, + file_partitions: Vec, +} + +impl MergeBarrierStream { + pub fn new( + input: SendableRecordBatchStream, + schema: SchemaRef, + survivors: BarrierSurvivorSet, + file_column: Arc, + ) -> Self { + // Always allocate for a null bucket at index 0; + let file_partitions = vec![MergeBarrierPartition::new(None)]; + + MergeBarrierStream { + schema, + state: State::Feed, + input, + file_column, + survivors, + file_partitions, + map: HashMap::new(), + } + } +} + +fn get_count(batch: &RecordBatch, column: &str) -> DataFusionResult { + batch + .column_by_name(column) + .map(|array| array.null_count()) + .ok_or_else(|| { + DataFusionError::External(Box::new(DeltaTableError::Generic( + "Required operation column is missing".to_string(), + ))) + }) +} + +impl Stream for MergeBarrierStream { + type Item = DataFusionResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + loop { + match self.state { + State::Feed => { + match self.input.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + let file_dictionary = get_path_column(&batch, &self.file_column)?; + + // For each record batch, the key for a file path is not stable. + // We can iterate through the dictionary and lookup the correspond string for each record and then lookup the correct `file_partition` for that value. + // However this approach exposes the cost of hashing so we want to minimize that as much as possible. + // A map from an arrow dictionary key to the correct index of `file_partition` is created for each batch that's processed. + // This ensures we only need to hash each file path at most once per batch. + let mut key_map = Vec::new(); + + for file_name in file_dictionary.values().into_iter() { + let key = match file_name { + Some(name) => { + if !self.map.contains_key(name) { + let key = self.file_partitions.len(); + let part_stream = + MergeBarrierPartition::new(Some(name.to_string())); + self.file_partitions.push(part_stream); + self.map.insert(name.to_string(), key); + } + // Safe unwrap due to the above + *self.map.get(name).unwrap() + } + None => 0, + }; + key_map.push(key) + } + + let mut indices: Vec<_> = (0..(self.file_partitions.len())) + .map(|_| UInt64Builder::with_capacity(batch.num_rows())) + .collect(); + + for (idx, key) in file_dictionary.keys().iter().enumerate() { + match key { + Some(value) => { + indices[key_map[value as usize]].append_value(idx as u64) + } + None => indices[0].append_value(idx as u64), + } + } + + let batches: Vec> = + indices + .into_iter() + .enumerate() + .filter_map(|(partition, mut indices)| { + let indices = indices.finish(); + (!indices.is_empty()).then_some((partition, indices)) + }) + .map(move |(partition, indices)| { + // Produce batches based on indices + let columns = batch + .columns() + .iter() + .map(|c| { + arrow::compute::take(c.as_ref(), &indices, None) + .map_err(|err| { + DataFusionError::ArrowError(err, None) + }) + }) + .collect::>>()?; + + // This unwrap is safe since the processed batched has the same schema + let batch = + RecordBatch::try_new(batch.schema(), columns).unwrap(); + + Ok((partition, batch)) + }) + .collect(); + + for batch in batches { + match batch { + Ok((partition, batch)) => { + self.file_partitions[partition].feed(batch)?; + } + Err(err) => { + self.state = State::Abort; + return Poll::Ready(Some(Err(err))); + } + } + } + + self.state = State::Drain; + continue; + } + Poll::Ready(Some(Err(err))) => { + self.state = State::Abort; + return Poll::Ready(Some(Err(err))); + } + Poll::Ready(None) => { + self.state = State::Finalize; + continue; + } + Poll::Pending => return Poll::Pending, + } + } + State::Drain => { + for part in &mut self.file_partitions { + if let Some(batch) = part.drain() { + return Poll::Ready(Some(Ok(batch))); + } + } + + self.state = State::Feed; + continue; + } + State::Finalize => { + for part in &mut self.file_partitions { + if let Some(batch) = part.drain() { + return Poll::Ready(Some(Ok(batch))); + } + } + + { + let mut lock = self.survivors.lock().map_err(|_| { + DataFusionError::External(Box::new(DeltaTableError::Generic( + "MergeBarrier mutex is poisoned".to_string(), + ))) + })?; + for part in &self.file_partitions { + match part.state { + PartitionBarrierState::Closed => {} + PartitionBarrierState::Open => { + if let Some(file_name) = &part.file_name { + lock.insert(file_name.to_owned()); + } + } + } + } + } + + self.state = State::Done; + continue; + } + State::Abort => return Poll::Ready(None), + State::Done => return Poll::Ready(None), + } + } + } + + fn size_hint(&self) -> (usize, Option) { + (0, self.input.size_hint().1) + } +} + +impl RecordBatchStream for MergeBarrierStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +#[derive(Debug, Hash, Eq, PartialEq)] +pub(crate) struct MergeBarrier { + pub input: LogicalPlan, + pub expr: Expr, + pub file_column: Arc, +} + +impl UserDefinedLogicalNodeCore for MergeBarrier { + fn name(&self) -> &str { + "MergeBarrier" + } + + fn inputs(&self) -> Vec<&datafusion_expr::LogicalPlan> { + vec![&self.input] + } + + fn schema(&self) -> &datafusion_common::DFSchemaRef { + self.input.schema() + } + + fn expressions(&self) -> Vec { + vec![self.expr.clone()] + } + + fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "MergeBarrier") + } + + fn from_template( + &self, + exprs: &[datafusion_expr::Expr], + inputs: &[datafusion_expr::LogicalPlan], + ) -> Self { + MergeBarrier { + input: inputs[0].clone(), + file_column: self.file_column.clone(), + expr: exprs[0].clone(), + } + } +} + +pub(crate) fn find_barrier_node(parent: &Arc) -> Option> { + //! Used to locate the physical Barrier Node after the planner converts the logical node + if parent.as_any().downcast_ref::().is_some() { + return Some(parent.to_owned()); + } + + for child in &parent.children() { + let res = find_barrier_node(child); + if res.is_some() { + return res; + } + } + + None +} + +#[cfg(test)] +mod tests { + use crate::operations::merge::MergeBarrierExec; + use crate::operations::merge::{ + TARGET_DELETE_COLUMN, TARGET_INSERT_COLUMN, TARGET_UPDATE_COLUMN, + }; + use arrow::datatypes::Schema as ArrowSchema; + use arrow_array::RecordBatch; + use arrow_array::StringArray; + use arrow_array::{DictionaryArray, UInt16Array}; + use arrow_schema::DataType as ArrowDataType; + use arrow_schema::Field; + use datafusion::assert_batches_sorted_eq; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; + use datafusion::physical_plan::memory::MemoryExec; + use datafusion::physical_plan::ExecutionPlan; + use datafusion_physical_expr::expressions::Column; + use futures::StreamExt; + use std::sync::Arc; + + use super::BarrierSurvivorSet; + + #[tokio::test] + async fn test_barrier() { + // Validate that files without modifications are dropped and that files with changes passthrough + // File 0: No Changes + // File 1: Contains an update + // File 2: Contains a delete + // null (id: 3): is a insert + + let schema = get_schema(); + let keys = UInt16Array::from(vec![Some(0), Some(1), Some(2), None]); + let values = StringArray::from(vec![Some("file0"), Some("file1"), Some("file2")]); + let dict = DictionaryArray::new(keys, Arc::new(values)); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["0", "1", "2", "3"])), + Arc::new(dict), + //insert column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + Some(false), + Some(false), + None, + ])), + //update column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + None, + Some(false), + Some(false), + ])), + //delete column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + Some(false), + None, + Some(false), + ])), + ], + ) + .unwrap(); + + let (actual, survivors) = execute(vec![batch]).await; + let expected = vec![ + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + "| id | __delta_rs_path | __delta_rs_target_insert | __delta_rs_target_update | __delta_rs_target_delete |", + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + "| 1 | file1 | false | | false |", + "| 2 | file2 | false | false | |", + "| 3 | | | false | false |", + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + ]; + assert_batches_sorted_eq!(&expected, &actual); + + let s = survivors.lock().unwrap(); + assert!(!s.contains(&"file0".to_string())); + assert!(s.contains(&"file1".to_string())); + assert!(s.contains(&"file2".to_string())); + assert_eq!(s.len(), 2); + } + + #[tokio::test] + async fn test_barrier_changing_indicies() { + // Validate implementation can handle different dictionary indicies between batches + + let schema = get_schema(); + let mut batches = vec![]; + + // Batch 1 + let keys = UInt16Array::from(vec![Some(0), Some(1)]); + let values = StringArray::from(vec![Some("file0"), Some("file1")]); + let dict = DictionaryArray::new(keys, Arc::new(values)); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["0", "1"])), + Arc::new(dict), + //insert column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + Some(false), + ])), + //update column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + Some(false), + ])), + //delete column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + Some(false), + ])), + ], + ) + .unwrap(); + batches.push(batch); + // Batch 2 + + let keys = UInt16Array::from(vec![Some(0), Some(1)]); + let values = StringArray::from(vec![Some("file1"), Some("file0")]); + let dict = DictionaryArray::new(keys, Arc::new(values)); + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["2", "3"])), + Arc::new(dict), + //insert column + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + Some(false), + ])), + //update column + Arc::new(arrow::array::BooleanArray::from(vec![None, Some(false)])), + //delete column + Arc::new(arrow::array::BooleanArray::from(vec![Some(false), None])), + ], + ) + .unwrap(); + batches.push(batch); + + let (actual, _survivors) = execute(batches).await; + let expected = vec! + [ + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + "| id | __delta_rs_path | __delta_rs_target_insert | __delta_rs_target_update | __delta_rs_target_delete |", + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + "| 0 | file0 | false | false | false |", + "| 1 | file1 | false | false | false |", + "| 2 | file1 | false | | false |", + "| 3 | file0 | false | false | |", + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + ]; + assert_batches_sorted_eq!(&expected, &actual); + } + + #[tokio::test] + async fn test_barrier_null_paths() { + // Arrow dictionaries are interesting since a null value can be either in the keys of the dict or in the values. + // Validate they can be processed without issue + + let schema = get_schema(); + let keys = UInt16Array::from(vec![Some(0), None, Some(1)]); + let values = StringArray::from(vec![Some("file1"), None]); + let dict = DictionaryArray::new(keys, Arc::new(values)); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["1", "2", "3"])), + Arc::new(dict), + Arc::new(arrow::array::BooleanArray::from(vec![ + Some(false), + None, + None, + ])), + Arc::new(arrow::array::BooleanArray::from(vec![false, false, false])), + Arc::new(arrow::array::BooleanArray::from(vec![false, false, false])), + ], + ) + .unwrap(); + + let (actual, _) = execute(vec![batch]).await; + let expected = vec![ + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + "| id | __delta_rs_path | __delta_rs_target_insert | __delta_rs_target_update | __delta_rs_target_delete |", + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + "| 2 | | | false | false |", + "| 3 | | | false | false |", + "+----+-----------------+--------------------------+--------------------------+--------------------------+", + ]; + assert_batches_sorted_eq!(&expected, &actual); + } + + async fn execute(input: Vec) -> (Vec, BarrierSurvivorSet) { + let schema = get_schema(); + let repartition = Arc::new(Column::new("__delta_rs_path", 2)); + let exec = Arc::new(MemoryExec::try_new(&[input], schema.clone(), None).unwrap()); + + let task_ctx = Arc::new(TaskContext::default()); + let merge = + MergeBarrierExec::new(exec, Arc::new("__delta_rs_path".to_string()), repartition); + + let survivors = merge.survivors(); + let coalsece = CoalesceBatchesExec::new(Arc::new(merge), 100); + let mut stream = coalsece.execute(0, task_ctx).unwrap(); + (vec![stream.next().await.unwrap().unwrap()], survivors) + } + + fn get_schema() -> Arc { + Arc::new(ArrowSchema::new(vec![ + Field::new("id", ArrowDataType::Utf8, true), + Field::new( + "__delta_rs_path", + ArrowDataType::Dictionary( + Box::new(ArrowDataType::UInt16), + Box::new(ArrowDataType::Utf8), + ), + true, + ), + Field::new(TARGET_INSERT_COLUMN, ArrowDataType::Boolean, true), + Field::new(TARGET_UPDATE_COLUMN, ArrowDataType::Boolean, true), + Field::new(TARGET_DELETE_COLUMN, ArrowDataType::Boolean, true), + ])) + } +} diff --git a/crates/deltalake-core/src/operations/merge.rs b/crates/core/src/operations/merge/mod.rs similarity index 64% rename from crates/deltalake-core/src/operations/merge.rs rename to crates/core/src/operations/merge/mod.rs index 433e9cda43..e5da42bf8f 100644 --- a/crates/deltalake-core/src/operations/merge.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -7,10 +7,6 @@ //! and specify additional predicates for finer control. The order of operations //! specified matter. See [`MergeBuilder`] for more information //! -//! *WARNING* The current implementation rewrites the entire delta table so only -//! use on small to medium sized tables. -//! Enhancements tracked at #850 -//! //! # Example //! ```rust ignore //! let table = open_table("../path/to/table")?; @@ -34,7 +30,7 @@ use std::collections::HashMap; use std::sync::Arc; -use std::time::{Instant, SystemTime, UNIX_EPOCH}; +use std::time::Instant; use async_trait::async_trait; use datafusion::datasource::provider_as_source; @@ -50,38 +46,50 @@ use datafusion::{ }, prelude::{DataFrame, SessionContext}, }; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Column, DFSchema, ScalarValue, TableReference}; +use datafusion_expr::expr::Placeholder; use datafusion_expr::{col, conditional_expressions::CaseBuilder, lit, when, Expr, JoinType}; use datafusion_expr::{ - Extension, LogicalPlan, LogicalPlanBuilder, UserDefinedLogicalNode, UNNAMED_TABLE, + BinaryExpr, Distinct, Extension, Filter, LogicalPlan, LogicalPlanBuilder, Operator, Projection, + UserDefinedLogicalNode, UNNAMED_TABLE, }; use futures::future::BoxFuture; +use itertools::Itertools; use parquet::file::properties::WriterProperties; use serde::Serialize; -use serde_json::Value; + +use self::barrier::{MergeBarrier, MergeBarrierExec}; use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; -use super::transaction::{commit, PROTOCOL}; +use super::transaction::{CommitProperties, PROTOCOL}; use crate::delta_datafusion::expr::{fmt_expr_to_sql, parse_predicate_expression}; use crate::delta_datafusion::logical::MetricObserver; use crate::delta_datafusion::physical::{find_metric_node, MetricObserverExec}; -use crate::delta_datafusion::{register_store, DeltaScanConfig, DeltaTableProvider}; -use crate::kernel::{Action, Remove}; +use crate::delta_datafusion::{ + execute_plan_to_batch, register_store, DeltaColumn, DeltaScanConfigBuilder, DeltaSessionConfig, + DeltaTableProvider, +}; +use crate::kernel::Action; use crate::logstore::LogStoreRef; +use crate::operations::merge::barrier::find_barrier_node; +use crate::operations::transaction::CommitBuilder; use crate::operations::write::write_execution_plan; use crate::protocol::{DeltaOperation, MergePredicate}; use crate::table::state::DeltaTableState; use crate::{DeltaResult, DeltaTable, DeltaTableError}; +mod barrier; + const SOURCE_COLUMN: &str = "__delta_rs_source"; const TARGET_COLUMN: &str = "__delta_rs_target"; const OPERATION_COLUMN: &str = "__delta_rs_operation"; const DELETE_COLUMN: &str = "__delta_rs_delete"; -const TARGET_INSERT_COLUMN: &str = "__delta_rs_target_insert"; -const TARGET_UPDATE_COLUMN: &str = "__delta_rs_target_update"; -const TARGET_DELETE_COLUMN: &str = "__delta_rs_target_delete"; -const TARGET_COPY_COLUMN: &str = "__delta_rs_target_copy"; +pub(crate) const TARGET_INSERT_COLUMN: &str = "__delta_rs_target_insert"; +pub(crate) const TARGET_UPDATE_COLUMN: &str = "__delta_rs_target_update"; +pub(crate) const TARGET_DELETE_COLUMN: &str = "__delta_rs_target_delete"; +pub(crate) const TARGET_COPY_COLUMN: &str = "__delta_rs_target_copy"; const SOURCE_COUNT_METRIC: &str = "num_source_rows"; const TARGET_COUNT_METRIC: &str = "num_target_rows"; @@ -118,8 +126,8 @@ pub struct MergeBuilder { state: Option, /// Properties passed to underlying parquet writer for when files are rewritten writer_properties: Option, - /// Additional metadata to be added to commit - app_metadata: Option>, + /// Additional information to add to the commit + commit_properties: CommitProperties, /// safe_cast determines how data types that do not match the underlying table are handled /// By default an error is returned safe_cast: bool, @@ -142,7 +150,7 @@ impl MergeBuilder { source_alias: None, target_alias: None, state: None, - app_metadata: None, + commit_properties: CommitProperties::default(), writer_properties: None, match_operations: Vec::new(), not_match_operations: Vec::new(), @@ -343,11 +351,8 @@ impl MergeBuilder { } /// Additional metadata to be added to commit info - pub fn with_metadata( - mut self, - metadata: impl IntoIterator, - ) -> Self { - self.app_metadata = Some(HashMap::from_iter(metadata)); + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; self } @@ -391,12 +396,12 @@ impl UpdateBuilder { /// How a column from the target table should be updated. /// In the match case the expression may contain both source and target columns. /// In the source not match case the expression may only contain target columns - pub fn update, E: Into>( + pub fn update, E: Into>( mut self, column: C, expression: E, ) -> Self { - self.updates.insert(column.into(), expression.into()); + self.updates.insert(column.into().into(), expression.into()); self } } @@ -419,8 +424,12 @@ impl InsertBuilder { /// Which values to insert into the target tables. If a target column is not /// specified then null is inserted. - pub fn set, E: Into>(mut self, column: C, expression: E) -> Self { - self.set.insert(column.into(), expression.into()); + pub fn set, E: Into>( + mut self, + column: C, + expression: E, + ) -> Self { + self.set.insert(column.into().into(), expression.into()); self } } @@ -569,11 +578,11 @@ struct MergeMetricExtensionPlanner {} impl ExtensionPlanner for MergeMetricExtensionPlanner { async fn plan_extension( &self, - _planner: &dyn PhysicalPlanner, + planner: &dyn PhysicalPlanner, node: &dyn UserDefinedLogicalNode, _logical_inputs: &[&LogicalPlan], physical_inputs: &[Arc], - _session_state: &SessionState, + session_state: &SessionState, ) -> DataFusionResult>> { if let Some(metric_observer) = node.as_any().downcast_ref::() { if metric_observer.id.eq(SOURCE_COUNT_ID) { @@ -642,10 +651,288 @@ impl ExtensionPlanner for MergeMetricExtensionPlanner { } } + if let Some(barrier) = node.as_any().downcast_ref::() { + let schema = barrier.input.schema(); + return Ok(Some(Arc::new(MergeBarrierExec::new( + physical_inputs.first().unwrap().clone(), + barrier.file_column.clone(), + planner.create_physical_expr(&barrier.expr, schema, session_state)?, + )))); + } + Ok(None) } } +/// Takes the predicate provided and does two things: +/// +/// 1. for any relations between a source column and a target column, if the target column is a +/// partition column, then replace source with a placeholder matching the name of the partition +/// columns +/// +/// 2. for any other relation with a source column, remove them. +/// +/// For example, for the predicate: +/// +/// `source.date = target.date and source.id = target.id and frob > 42` +/// +/// where `date` is a partition column, would result in the expr: +/// +/// `$date = target.date and frob > 42` +/// +/// This leaves us with a predicate that we can push into delta scan after expanding it out to +/// a conjunction between the disinct partitions in the source input. +/// +/// TODO: A futher improvement here might be for non-partition columns to be replaced with min/max +/// checks, so the above example could become: +/// +/// `$date = target.date and target.id between 12345 and 99999 and frob > 42` +fn generalize_filter( + predicate: Expr, + partition_columns: &Vec, + source_name: &TableReference, + target_name: &TableReference, + placeholders: &mut HashMap, +) -> Option { + #[derive(Debug)] + enum ReferenceTableCheck { + HasReference(String), + NoReference, + Unknown, + } + impl ReferenceTableCheck { + fn has_reference(&self) -> bool { + matches!(self, ReferenceTableCheck::HasReference(_)) + } + } + fn references_table(expr: &Expr, table: &TableReference) -> ReferenceTableCheck { + let res = match expr { + Expr::Alias(alias) => references_table(&alias.expr, table), + Expr::Column(col) => col + .relation + .as_ref() + .map(|rel| { + if rel == table { + ReferenceTableCheck::HasReference(col.name.to_owned()) + } else { + ReferenceTableCheck::NoReference + } + }) + .unwrap_or(ReferenceTableCheck::NoReference), + Expr::Negative(neg) => references_table(neg, table), + Expr::Cast(cast) => references_table(&cast.expr, table), + Expr::TryCast(try_cast) => references_table(&try_cast.expr, table), + Expr::ScalarFunction(func) => { + if func.args.len() == 1 { + references_table(&func.args[0], table) + } else { + ReferenceTableCheck::Unknown + } + } + Expr::IsNull(inner) => references_table(inner, table), + Expr::Literal(_) => ReferenceTableCheck::NoReference, + _ => ReferenceTableCheck::Unknown, + }; + res + } + + match predicate { + Expr::BinaryExpr(binary) => { + if references_table(&binary.right, source_name).has_reference() { + if let ReferenceTableCheck::HasReference(left_target) = + references_table(&binary.left, target_name) + { + if partition_columns.contains(&left_target) { + let placeholder_name = format!("{left_target}_{}", placeholders.len()); + + let placeholder = Expr::Placeholder(datafusion_expr::expr::Placeholder { + id: placeholder_name.clone(), + data_type: None, + }); + let replaced = Expr::BinaryExpr(BinaryExpr { + left: binary.left, + op: binary.op, + right: placeholder.into(), + }); + + placeholders.insert(placeholder_name, *binary.right); + + return Some(replaced); + } + } + return None; + } + if references_table(&binary.left, source_name).has_reference() { + if let ReferenceTableCheck::HasReference(right_target) = + references_table(&binary.right, target_name) + { + if partition_columns.contains(&right_target) { + let placeholder_name = format!("{right_target}_{}", placeholders.len()); + + let placeholder = Expr::Placeholder(datafusion_expr::expr::Placeholder { + id: placeholder_name.clone(), + data_type: None, + }); + let replaced = Expr::BinaryExpr(BinaryExpr { + right: binary.right, + op: binary.op, + left: placeholder.into(), + }); + + placeholders.insert(placeholder_name, *binary.left); + + return Some(replaced); + } + } + return None; + } + + let left = generalize_filter( + *binary.left, + partition_columns, + source_name, + target_name, + placeholders, + ); + let right = generalize_filter( + *binary.right, + partition_columns, + source_name, + target_name, + placeholders, + ); + + match (left, right) { + (None, None) => None, + (None, Some(one_side)) | (Some(one_side), None) => { + // in the case of an AND clause, it's safe to generalize the filter down to just one side of the AND. + // this is because this filter will be more permissive than the actual predicate, so we know that + // we will catch all data that could be matched by the predicate. For OR this is not the case - we + // could potentially eliminate one side of the predicate and the filter would only match half the + // cases that would have satisfied the match predicate. + match binary.op { + Operator::And => Some(one_side), + Operator::Or => None, + _ => None, + } + } + (Some(l), Some(r)) => Expr::BinaryExpr(BinaryExpr { + left: l.into(), + op: binary.op, + right: r.into(), + }) + .into(), + } + } + other => match references_table(&other, source_name) { + ReferenceTableCheck::HasReference(col) => { + let placeholder_name = format!("{col}_{}", placeholders.len()); + + let placeholder = Expr::Placeholder(datafusion_expr::expr::Placeholder { + id: placeholder_name.clone(), + data_type: None, + }); + + placeholders.insert(placeholder_name, other); + + Some(placeholder) + } + ReferenceTableCheck::NoReference => Some(other), + ReferenceTableCheck::Unknown => None, + }, + } +} + +fn replace_placeholders(expr: Expr, placeholders: &HashMap) -> Expr { + expr.transform(&|expr| match expr { + Expr::Placeholder(Placeholder { id, .. }) => { + let value = placeholders[&id].clone(); + // Replace the placeholder with the value + Ok(Transformed::Yes(Expr::Literal(value))) + } + _ => Ok(Transformed::No(expr)), + }) + .unwrap() +} + +async fn try_construct_early_filter( + join_predicate: Expr, + table_snapshot: &DeltaTableState, + session_state: &SessionState, + source: &LogicalPlan, + source_name: &TableReference<'_>, + target_name: &TableReference<'_>, +) -> DeltaResult> { + let table_metadata = table_snapshot.metadata(); + let partition_columns = &table_metadata.partition_columns; + + if partition_columns.is_empty() { + return Ok(None); + } + + let mut placeholders = HashMap::default(); + + match generalize_filter( + join_predicate, + partition_columns, + source_name, + target_name, + &mut placeholders, + ) { + None => Ok(None), + Some(filter) => { + if placeholders.is_empty() { + // if we haven't recognised any partition-based predicates in the join predicate, return our reduced filter + Ok(Some(filter)) + } else { + // if we have some recognised partitions, then discover the distinct set of partitions in the source data and + // make a new filter, which expands out the placeholders for each distinct partition (and then OR these together) + let distinct_partitions = LogicalPlan::Distinct(Distinct::All( + LogicalPlan::Projection(Projection::try_new( + placeholders + .into_iter() + .map(|(alias, expr)| expr.alias(alias)) + .collect_vec(), + source.clone().into(), + )?) + .into(), + )); + + let execution_plan = session_state + .create_physical_plan(&distinct_partitions) + .await?; + + let items = execute_plan_to_batch(session_state, execution_plan).await?; + + let placeholder_names = items + .schema() + .fields() + .iter() + .map(|f| f.name().to_owned()) + .collect_vec(); + + let expr = (0..items.num_rows()) + .map(|i| { + let replacements = placeholder_names + .iter() + .map(|placeholder| { + let col = items.column_by_name(placeholder).unwrap(); + let value = ScalarValue::try_from_array(col, i)?; + DeltaResult::Ok((placeholder.to_owned(), value)) + }) + .try_collect()?; + Ok(replace_placeholders(filter.clone(), &replacements)) + }) + .collect::>>()? + .into_iter() + .reduce(Expr::or); + + Ok(expr) + } + } + } +} + #[allow(clippy::too_many_arguments)] async fn execute( predicate: Expression, @@ -654,18 +941,18 @@ async fn execute( snapshot: &DeltaTableState, state: SessionState, writer_properties: Option, - app_metadata: Option>, + mut commit_properties: CommitProperties, safe_cast: bool, source_alias: Option, target_alias: Option, match_operations: Vec, not_match_target_operations: Vec, not_match_source_operations: Vec, -) -> DeltaResult<((Vec, i64), MergeMetrics)> { +) -> DeltaResult<((Vec, i64, Option), MergeMetrics)> { let mut metrics = MergeMetrics::default(); let exec_start = Instant::now(); - let current_metadata = snapshot.metadata().ok_or(DeltaTableError::NoMetadata)?; + let current_metadata = snapshot.metadata(); // TODO: Given the join predicate, remove any expression that involve the // source table and keep expressions that only involve the target table. @@ -687,47 +974,88 @@ async fn execute( }; // This is only done to provide the source columns with a correct table reference. Just renaming the columns does not work - let source = - LogicalPlanBuilder::scan(source_name, provider_as_source(source.into_view()), None)? - .build()?; + let source = LogicalPlanBuilder::scan( + source_name.clone(), + provider_as_source(source.into_view()), + None, + )? + .build()?; let source = LogicalPlan::Extension(Extension { node: Arc::new(MetricObserver { id: SOURCE_COUNT_ID.into(), input: source, + enable_pushdown: false, }), }); - let source = DataFrame::new(state.clone(), source); - let source = source.with_column(SOURCE_COLUMN, lit(true))?; + let scan_config = DeltaScanConfigBuilder::default() + .with_file_column(true) + .build(snapshot)?; + + let file_column = Arc::new(scan_config.file_column_name.clone().unwrap()); let target_provider = Arc::new(DeltaTableProvider::try_new( snapshot.clone(), log_store.clone(), - DeltaScanConfig::default(), + scan_config, )?); + let target_provider = provider_as_source(target_provider); - let target = LogicalPlanBuilder::scan(target_name, target_provider, None)?.build()?; + let target = LogicalPlanBuilder::scan(target_name.clone(), target_provider, None)?.build()?; + + let source_schema = source.schema(); + let target_schema = target.schema(); + let join_schema_df = build_join_schema(source_schema, target_schema, &JoinType::Full)?; + let predicate = match predicate { + Expression::DataFusion(expr) => expr, + Expression::String(s) => parse_predicate_expression(&join_schema_df, s, &state)?, + }; + + let state = state.with_query_planner(Arc::new(MergePlanner {})); - // TODO: This is here to prevent predicate pushdowns. In the future we can replace this node to allow pushdowns depending on which operations are being used. + let target = { + // Attempt to construct an early filter that we can apply to the Add action list and the delta scan. + // In the case where there are partition columns in the join predicate, we can scan the source table + // to get the distinct list of partitions affected and constrain the search to those. + + if !not_match_source_operations.is_empty() { + // It's only worth trying to create an early filter where there are no `when_not_matched_source` operators, since + // that implies a full scan + target + } else if let Some(filter) = try_construct_early_filter( + predicate.clone(), + snapshot, + &state, + &source, + &source_name, + &target_name, + ) + .await? + { + LogicalPlan::Filter(Filter::try_new(filter, target.into())?) + } else { + target + } + }; + + let source = DataFrame::new(state.clone(), source); + let source = source.with_column(SOURCE_COLUMN, lit(true))?; + + // Not match operations imply a full scan of the target table is required + let enable_pushdown = + not_match_source_operations.is_empty() && not_match_target_operations.is_empty(); let target = LogicalPlan::Extension(Extension { node: Arc::new(MetricObserver { id: TARGET_COUNT_ID.into(), input: target, + enable_pushdown, }), }); let target = DataFrame::new(state.clone(), target); let target = target.with_column(TARGET_COLUMN, lit(true))?; - let source_schema = source.schema(); - let target_schema = target.schema(); - let join_schema_df = build_join_schema(source_schema, target_schema, &JoinType::Full)?; - let predicate = match predicate { - Expression::DataFusion(expr) => expr, - Expression::String(s) => parse_predicate_expression(&join_schema_df, s, &state)?, - }; - let join = source.join(target, JoinType::Full, &[], &[], Some(predicate.clone()))?; let join_schema_df = join.schema().to_owned(); @@ -853,10 +1181,10 @@ async fn execute( let projection = join.with_column(OPERATION_COLUMN, case)?; - let mut new_columns = projection; + let mut new_columns = vec![]; let mut write_projection = Vec::new(); - for delta_field in snapshot.schema().unwrap().fields() { + for delta_field in snapshot.schema().fields() { let mut when_expr = Vec::with_capacity(operations_size); let mut then_expr = Vec::with_capacity(operations_size); @@ -888,8 +1216,11 @@ async fn execute( .end()?; let name = "__delta_rs_c_".to_owned() + delta_field.name(); - write_projection.push(col(name.clone()).alias(delta_field.name())); - new_columns = new_columns.with_column(&name, case)?; + write_projection.push( + Expr::Column(Column::from_qualified_name_ignore_case(name.clone())) + .alias(delta_field.name()), + ); + new_columns.push((name, case)); } let mut insert_when = Vec::with_capacity(ops.len()); @@ -965,22 +1296,56 @@ async fn execute( .end() } - new_columns = new_columns.with_column(DELETE_COLUMN, build_case(delete_when, delete_then)?)?; - new_columns = - new_columns.with_column(TARGET_INSERT_COLUMN, build_case(insert_when, insert_then)?)?; - new_columns = - new_columns.with_column(TARGET_UPDATE_COLUMN, build_case(update_when, update_then)?)?; - new_columns = new_columns.with_column( - TARGET_DELETE_COLUMN, + new_columns.push(( + DELETE_COLUMN.to_owned(), + build_case(delete_when, delete_then)?, + )); + new_columns.push(( + TARGET_INSERT_COLUMN.to_owned(), + build_case(insert_when, insert_then)?, + )); + new_columns.push(( + TARGET_UPDATE_COLUMN.to_owned(), + build_case(update_when, update_then)?, + )); + new_columns.push(( + TARGET_DELETE_COLUMN.to_owned(), build_case(target_delete_when, target_delete_then)?, - )?; - new_columns = new_columns.with_column(TARGET_COPY_COLUMN, build_case(copy_when, copy_then)?)?; + )); + new_columns.push(( + TARGET_COPY_COLUMN.to_owned(), + build_case(copy_when, copy_then)?, + )); + + let new_columns = { + let plan = projection.into_unoptimized_plan(); + let mut fields: Vec = plan + .schema() + .fields() + .iter() + .map(|f| col(f.qualified_column())) + .collect(); + + fields.extend(new_columns.into_iter().map(|(name, ex)| ex.alias(name))); + + LogicalPlanBuilder::from(plan).project(fields)?.build()? + }; + + let distrbute_expr = col(file_column.as_str()); + + let merge_barrier = LogicalPlan::Extension(Extension { + node: Arc::new(MergeBarrier { + input: new_columns, + expr: distrbute_expr, + file_column, + }), + }); - let new_columns = new_columns.into_optimized_plan()?; let operation_count = LogicalPlan::Extension(Extension { node: Arc::new(MetricObserver { id: OUTPUT_COUNT_ID.into(), - input: new_columns, + input: merge_barrier, + enable_pushdown: false, }), }); @@ -988,21 +1353,21 @@ async fn execute( let filtered = operation_count.filter(col(DELETE_COLUMN).is_false())?; let project = filtered.select(write_projection)?; - let optimized = &project.into_optimized_plan()?; + let merge_final = &project.into_unoptimized_plan(); - let state = state.with_query_planner(Arc::new(MergePlanner {})); - let write = state.create_physical_plan(optimized).await?; + let write = state.create_physical_plan(merge_final).await?; let err = || DeltaTableError::Generic("Unable to locate expected metric node".into()); let source_count = find_metric_node(SOURCE_COUNT_ID, &write).ok_or_else(err)?; let op_count = find_metric_node(OUTPUT_COUNT_ID, &write).ok_or_else(err)?; + let barrier = find_barrier_node(&write).ok_or_else(err)?; // write projected records let table_partition_cols = current_metadata.partition_columns.clone(); let rewrite_start = Instant::now(); let add_actions = write_execution_plan( - snapshot, + Some(snapshot), state.clone(), write, table_partition_cols.clone(), @@ -1011,37 +1376,30 @@ async fn execute( None, writer_properties, safe_cast, - false, + None, ) .await?; metrics.rewrite_time_ms = Instant::now().duration_since(rewrite_start).as_millis() as u64; - let deletion_timestamp = SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as i64; - - let mut actions: Vec = add_actions.into_iter().map(Action::Add).collect(); + let mut actions: Vec = add_actions.clone(); metrics.num_target_files_added = actions.len(); - for action in snapshot.files() { - metrics.num_target_files_removed += 1; - actions.push(Action::Remove(Remove { - path: action.path.clone(), - deletion_timestamp: Some(deletion_timestamp), - data_change: true, - extended_file_metadata: Some(true), - partition_values: Some(action.partition_values.clone()), - deletion_vector: action.deletion_vector.clone(), - size: Some(action.size), - tags: None, - base_row_id: action.base_row_id, - default_row_commit_version: action.default_row_commit_version, - })) - } + let survivors = barrier + .as_any() + .downcast_ref::() + .unwrap() + .survivors(); - let mut version = snapshot.version(); + { + let lock = survivors.lock().unwrap(); + for action in snapshot.log_data() { + if lock.contains(action.path().as_ref()) { + metrics.num_target_files_removed += 1; + actions.push(action.remove_action(true).into()); + } + } + } let source_count_metrics = source_count.metrics().unwrap(); let target_count_metrics = op_count.metrics().unwrap(); @@ -1060,25 +1418,36 @@ async fn execute( metrics.execution_time_ms = Instant::now().duration_since(exec_start).as_millis() as u64; + let app_metadata = &mut commit_properties.app_metadata; + app_metadata.insert("readVersion".to_owned(), snapshot.version().into()); + if let Ok(map) = serde_json::to_value(&metrics) { + app_metadata.insert("operationMetrics".to_owned(), map); + } + // Do not make a commit when there are zero updates to the state - if !actions.is_empty() { - let operation = DeltaOperation::Merge { - predicate: Some(fmt_expr_to_sql(&predicate)?), - matched_predicates: match_operations, - not_matched_predicates: not_match_target_operations, - not_matched_by_source_predicates: not_match_source_operations, - }; - version = commit( - log_store.as_ref(), - &actions, - operation, - snapshot, - app_metadata, - ) - .await?; + let operation = DeltaOperation::Merge { + predicate: Some(fmt_expr_to_sql(&predicate)?), + matched_predicates: match_operations, + not_matched_predicates: not_match_target_operations, + not_matched_by_source_predicates: not_match_source_operations, + }; + + if actions.is_empty() { + return Ok(((actions, snapshot.version(), None), metrics)); } - Ok(((actions, version), metrics)) + let commit = CommitBuilder::from(commit_properties) + .with_actions(actions) + .build(Some(snapshot), log_store.clone(), operation)? + .await?; + Ok(( + ( + commit.data.actions, + commit.version, + Some(commit.data.operation), + ), + metrics, + )) } // TODO: Abstract MergePlanner into DeltaPlanner to support other delta operations in the future. @@ -1108,11 +1477,10 @@ impl std::future::IntoFuture for MergeBuilder { let mut this = self; Box::pin(async move { - PROTOCOL.can_write_to(&this.snapshot)?; + PROTOCOL.can_write_to(&this.snapshot.snapshot)?; let state = this.state.unwrap_or_else(|| { - //TODO: Datafusion's Hashjoin has some memory issues. Running with all cores results in a OoM. Can be removed when upstream improvemetns are made. - let config = SessionConfig::new().with_target_partitions(1); + let config: SessionConfig = DeltaSessionConfig::default().into(); let session = SessionContext::new_with_config(config); // If a user provides their own their DF state then they must register the store themselves @@ -1121,14 +1489,14 @@ impl std::future::IntoFuture for MergeBuilder { session.state() }); - let ((actions, version), metrics) = execute( + let ((actions, version, operation), metrics) = execute( this.predicate, this.source, this.log_store.clone(), &this.snapshot, state, this.writer_properties, - this.app_metadata, + this.commit_properties, this.safe_cast, this.source_alias, this.target_alias, @@ -1138,8 +1506,9 @@ impl std::future::IntoFuture for MergeBuilder { ) .await?; - this.snapshot - .merge(DeltaTableState::from_actions(actions, version)?, true, true); + if let Some(op) = &operation { + this.snapshot.merge(actions, op, version)?; + } let table = DeltaTable::new_with_state(this.log_store, this.snapshot); Ok((table, metrics)) @@ -1149,6 +1518,11 @@ impl std::future::IntoFuture for MergeBuilder { #[cfg(test)] mod tests { + use crate::kernel::DataType; + use crate::kernel::PrimitiveType; + use crate::kernel::StructField; + use crate::operations::merge::generalize_filter; + use crate::operations::merge::try_construct_early_filter; use crate::operations::DeltaOps; use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; @@ -1159,12 +1533,25 @@ mod tests { use crate::DeltaTable; use arrow::datatypes::Schema as ArrowSchema; use arrow::record_batch::RecordBatch; + use arrow_schema::DataType as ArrowDataType; + use arrow_schema::Field; use datafusion::assert_batches_sorted_eq; + use datafusion::datasource::provider_as_source; use datafusion::prelude::DataFrame; use datafusion::prelude::SessionContext; + use datafusion_common::Column; + use datafusion_common::ScalarValue; + use datafusion_common::TableReference; use datafusion_expr::col; + use datafusion_expr::expr::Placeholder; use datafusion_expr::lit; + use datafusion_expr::Expr; + use datafusion_expr::LogicalPlanBuilder; + use datafusion_expr::Operator; + use itertools::Itertools; use serde_json::json; + use std::collections::HashMap; + use std::ops::Neg; use std::sync::Arc; use super::MergeMetrics; @@ -1182,6 +1569,7 @@ mod tests { table } + // TODO(ion): property keys are not passed through or translated as table features.. fix this as well #[tokio::test] async fn test_merge_when_delta_table_is_append_only() { let schema = get_arrow_schema(&None); @@ -1193,6 +1581,8 @@ mod tests { .merge(merge_source(schema), col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") + .when_not_matched_by_source_delete(|delete| delete) + .unwrap() .await .expect_err("Remove action is included when Delta table is append-only. Should error"); } @@ -1244,14 +1634,14 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); (table, merge_source(schema)) } async fn assert_merge(table: DeltaTable, metrics: MergeMetrics) { assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 1); + assert!(table.get_files_count() >= 1); assert!(metrics.num_target_files_added >= 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); @@ -1280,7 +1670,7 @@ mod tests { async fn test_merge() { let (table, source) = setup().await; - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -1307,7 +1697,7 @@ mod tests { .unwrap(); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1332,7 +1722,7 @@ mod tests { // Also validates that update and set operations can contain the target alias let (table, source) = setup().await; - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, "target.id = source.id") .with_source_alias("source") .with_target_alias("target") @@ -1359,7 +1749,7 @@ mod tests { .unwrap(); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1493,13 +1883,13 @@ mod tests { #[tokio::test] async fn test_merge_partitions() { - /* Validate the join predicate works with partition columns */ + /* Validate the join predicate works with table partitions */ let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1555,7 +1945,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 3); + assert!(table.get_files_count() >= 3); assert!(metrics.num_target_files_added >= 3); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 1); @@ -1581,6 +1971,78 @@ mod tests { assert_batches_sorted_eq!(&expected, &actual); } + #[tokio::test] + async fn test_merge_partitions_skipping() { + /* Validate the join predicate can be used for skipping partitions */ + let schema = get_arrow_schema(&None); + let table = setup_table(Some(vec!["id"])).await; + + let table = write_data(table, &schema).await; + assert_eq!(table.version(), 1); + assert_eq!(table.get_files_count(), 4); + + let ctx = SessionContext::new(); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["B", "C", "X"])), + Arc::new(arrow::array::Int32Array::from(vec![999, 999, 999])), + Arc::new(arrow::array::StringArray::from(vec![ + "2023-07-04", + "2023-07-04", + "2023-07-04", + ])), + ], + ) + .unwrap(); + let source = ctx.read_batch(batch).unwrap(); + + let (table, metrics) = DeltaOps(table) + .merge(source, col("target.id").eq(col("source.id"))) + .with_source_alias("source") + .with_target_alias("target") + .when_matched_update(|update| { + update + .update("value", col("source.value")) + .update("modified", col("source.modified")) + }) + .unwrap() + .when_not_matched_insert(|insert| { + insert + .set("id", col("source.id")) + .set("value", col("source.value")) + .set("modified", col("source.modified")) + }) + .unwrap() + .await + .unwrap(); + + assert_eq!(table.version(), 2); + assert!(table.get_files_count() >= 3); + assert_eq!(metrics.num_target_files_added, 3); + assert_eq!(metrics.num_target_files_removed, 2); + assert_eq!(metrics.num_target_rows_copied, 0); + assert_eq!(metrics.num_target_rows_updated, 2); + assert_eq!(metrics.num_target_rows_inserted, 1); + assert_eq!(metrics.num_target_rows_deleted, 0); + assert_eq!(metrics.num_output_rows, 3); + assert_eq!(metrics.num_source_rows, 3); + + let expected = vec![ + "+-------+------------+----+", + "| value | modified | id |", + "+-------+------------+----+", + "| 1 | 2021-02-01 | A |", + "| 100 | 2021-02-02 | D |", + "| 999 | 2023-07-04 | B |", + "| 999 | 2023-07-04 | C |", + "| 999 | 2023-07-04 | X |", + "+-------+------------+----+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } + #[tokio::test] async fn test_merge_delete_matched() { // Validate behaviours of match delete @@ -1590,7 +2052,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1608,7 +2070,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -1618,8 +2080,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 2); - assert!(metrics.num_target_files_added >= 2); + assert!(table.get_files_count() >= 2); + assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 2); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1629,8 +2091,13 @@ mod tests { assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); + let extra_info = last_commit.info.clone(); + assert_eq!( + extra_info["operationMetrics"], + serde_json::to_value(&metrics).unwrap() + ); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( parameters["matchedPredicates"], @@ -1654,7 +2121,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1672,7 +2139,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -1682,18 +2149,18 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 2); - assert!(metrics.num_target_files_added >= 2); - assert_eq!(metrics.num_target_files_removed, 2); - assert_eq!(metrics.num_target_rows_copied, 3); + assert!(table.get_files_count() >= 2); + assert_eq!(metrics.num_target_files_added, 1); + assert_eq!(metrics.num_target_files_removed, 1); + assert_eq!(metrics.num_target_rows_copied, 1); assert_eq!(metrics.num_target_rows_updated, 0); assert_eq!(metrics.num_target_rows_inserted, 0); assert_eq!(metrics.num_target_rows_deleted, 1); - assert_eq!(metrics.num_output_rows, 3); + assert_eq!(metrics.num_output_rows, 1); assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1723,7 +2190,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1741,7 +2208,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -1751,7 +2218,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 2); @@ -1762,7 +2229,7 @@ mod tests { assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1786,7 +2253,7 @@ mod tests { let table = write_data(table, &schema).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1804,7 +2271,7 @@ mod tests { .unwrap(); let source = ctx.read_batch(batch).unwrap(); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") .with_target_alias("target") @@ -1816,17 +2283,17 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(metrics.num_target_files_added >= 2); - assert_eq!(metrics.num_target_files_removed, 2); - assert_eq!(metrics.num_target_rows_copied, 3); + assert!(metrics.num_target_files_added == 1); + assert_eq!(metrics.num_target_files_removed, 1); + assert_eq!(metrics.num_target_rows_copied, 1); assert_eq!(metrics.num_target_rows_updated, 0); assert_eq!(metrics.num_target_rows_inserted, 0); assert_eq!(metrics.num_target_rows_deleted, 1); - assert_eq!(metrics.num_output_rows, 3); + assert_eq!(metrics.num_output_rows, 1); assert_eq!(metrics.num_source_rows, 3); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( @@ -1853,7 +2320,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 0); + assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( @@ -1897,7 +2364,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 1); - assert!(table.get_file_uris().count() >= 2); + assert!(table.get_files_count() >= 2); assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 0); assert_eq!(metrics.num_target_rows_copied, 0); @@ -1919,4 +2386,356 @@ mod tests { let actual = get_data(&table).await; assert_batches_sorted_eq!(&expected, &actual); } + + #[tokio::test] + async fn test_merge_case_sensitive() { + let schema = vec![ + StructField::new( + "Id".to_string(), + DataType::Primitive(PrimitiveType::String), + true, + ), + StructField::new( + "vAlue".to_string(), + DataType::Primitive(PrimitiveType::Integer), + true, + ), + StructField::new( + "mOdifieD".to_string(), + DataType::Primitive(PrimitiveType::String), + true, + ), + ]; + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("Id", ArrowDataType::Utf8, true), + Field::new("vAlue", ArrowDataType::Int32, true), + Field::new("mOdifieD", ArrowDataType::Utf8, true), + ])); + + let table = DeltaOps::new_in_memory() + .create() + .with_columns(schema) + .await + .unwrap(); + + let ctx = SessionContext::new(); + let batch = RecordBatch::try_new( + Arc::clone(&arrow_schema.clone()), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["B", "C", "X"])), + Arc::new(arrow::array::Int32Array::from(vec![10, 20, 30])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2023-07-04", + "2023-07-04", + ])), + ], + ) + .unwrap(); + let source = ctx.read_batch(batch).unwrap(); + + let table = write_data(table, &arrow_schema).await; + assert_eq!(table.version(), 1); + assert_eq!(table.get_files_count(), 1); + + let (table, _metrics) = DeltaOps(table) + .merge(source, "target.Id = source.Id") + .with_source_alias("source") + .with_target_alias("target") + .when_not_matched_insert(|insert| { + insert + .set("Id", "source.Id") + .set("vAlue", "source.vAlue + 1") + .set("mOdifieD", "source.mOdifieD") + }) + .unwrap() + .await + .unwrap(); + + let expected = vec![ + "+----+-------+------------+", + "| Id | vAlue | mOdifieD |", + "+----+-------+------------+", + "| A | 1 | 2021-02-01 |", + "| B | 10 | 2021-02-01 |", + "| C | 10 | 2021-02-02 |", + "| D | 100 | 2021-02-02 |", + "| X | 31 | 2023-07-04 |", + "+----+-------+------------+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } + + #[tokio::test] + async fn test_generalize_filter_with_partitions() { + let source = TableReference::parse_str("source"); + let target = TableReference::parse_str("target"); + + let parsed_filter = col(Column::new(source.clone().into(), "id")) + .eq(col(Column::new(target.clone().into(), "id"))); + + let mut placeholders = HashMap::default(); + + let generalized = generalize_filter( + parsed_filter, + &vec!["id".to_owned()], + &source, + &target, + &mut placeholders, + ) + .unwrap(); + + let expected_filter = Expr::Placeholder(Placeholder { + id: "id_0".to_owned(), + data_type: None, + }) + .eq(col(Column::new(target.clone().into(), "id"))); + + assert_eq!(generalized, expected_filter); + } + + #[tokio::test] + async fn test_generalize_filter_with_partitions_nulls() { + let source = TableReference::parse_str("source"); + let target = TableReference::parse_str("target"); + + let source_id = col(Column::new(source.clone().into(), "id")); + let target_id = col(Column::new(target.clone().into(), "id")); + + // source.id = target.id OR (source.id is null and target.id is null) + let parsed_filter = (source_id.clone().eq(target_id.clone())) + .or(source_id.clone().is_null().and(target_id.clone().is_null())); + + let mut placeholders = HashMap::default(); + + let generalized = generalize_filter( + parsed_filter, + &vec!["id".to_owned()], + &source, + &target, + &mut placeholders, + ) + .unwrap(); + + // id_1 = target.id OR (id_2 and target.id is null) + let expected_filter = Expr::Placeholder(Placeholder { + id: "id_0".to_owned(), + data_type: None, + }) + .eq(target_id.clone()) + .or(Expr::Placeholder(Placeholder { + id: "id_1".to_owned(), + data_type: None, + }) + .and(target_id.clone().is_null())); + + assert!(placeholders.len() == 2); + + let captured_expressions = placeholders.values().collect_vec(); + + assert!(captured_expressions.contains(&&source_id)); + assert!(captured_expressions.contains(&&source_id.is_null())); + + assert_eq!(generalized, expected_filter); + } + + #[tokio::test] + async fn test_generalize_filter_with_partitions_captures_expression() { + // Check that when generalizing the filter, the placeholder map captures the expression needed to make the statement the same + // when the distinct values are substitiuted in + let source = TableReference::parse_str("source"); + let target = TableReference::parse_str("target"); + + let parsed_filter = col(Column::new(source.clone().into(), "id")) + .neg() + .eq(col(Column::new(target.clone().into(), "id"))); + + let mut placeholders = HashMap::default(); + + let generalized = generalize_filter( + parsed_filter, + &vec!["id".to_owned()], + &source, + &target, + &mut placeholders, + ) + .unwrap(); + + let expected_filter = Expr::Placeholder(Placeholder { + id: "id_0".to_owned(), + data_type: None, + }) + .eq(col(Column::new(target.clone().into(), "id"))); + + assert_eq!(generalized, expected_filter); + + assert_eq!(placeholders.len(), 1); + + let placeholder_expr = &placeholders["id_0"]; + + let expected_placeholder = col(Column::new(source.clone().into(), "id")).neg(); + + assert_eq!(placeholder_expr, &expected_placeholder); + } + + #[tokio::test] + async fn test_generalize_filter_keeps_static_target_references() { + let source = TableReference::parse_str("source"); + let target = TableReference::parse_str("target"); + + // source.id = target.id and target.id = 'C' + let parsed_filter = col(Column::new(source.clone().into(), "id")) + .eq(col(Column::new(target.clone().into(), "id"))) + .and(col(Column::new(target.clone().into(), "id")).eq(lit("C"))); + + let mut placeholders = HashMap::default(); + + let generalized = generalize_filter( + parsed_filter, + &vec!["id".to_owned()], + &source, + &target, + &mut placeholders, + ) + .unwrap(); + + // id_0 = target.id and target.id = 'C' + let expected_filter = Expr::Placeholder(Placeholder { + id: "id_0".to_owned(), + data_type: None, + }) + .eq(col(Column::new(target.clone().into(), "id"))) + .and(col(Column::new(target.clone().into(), "id")).eq(lit("C"))); + + assert_eq!(generalized, expected_filter); + } + + #[tokio::test] + async fn test_generalize_filter_removes_source_references() { + let source = TableReference::parse_str("source"); + let target = TableReference::parse_str("target"); + + let parsed_filter = col(Column::new(source.clone().into(), "id")) + .eq(col(Column::new(target.clone().into(), "id"))) + .and(col(Column::new(source.clone().into(), "id")).eq(lit("C"))); + + let mut placeholders = HashMap::default(); + + let generalized = generalize_filter( + parsed_filter, + &vec!["id".to_owned()], + &source, + &target, + &mut placeholders, + ) + .unwrap(); + + let expected_filter = Expr::Placeholder(Placeholder { + id: "id_0".to_owned(), + data_type: None, + }) + .eq(col(Column::new(target.clone().into(), "id"))); + + assert_eq!(generalized, expected_filter); + } + + #[tokio::test] + async fn test_try_construct_early_filter_with_partitions_expands() { + let schema = get_arrow_schema(&None); + let table = setup_table(Some(vec!["id"])).await; + + assert_eq!(table.version(), 0); + assert_eq!(table.get_files_count(), 0); + + let ctx = SessionContext::new(); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["B", "C", "X"])), + Arc::new(arrow::array::Int32Array::from(vec![10, 20, 30])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2023-07-04", + "2023-07-04", + ])), + ], + ) + .unwrap(); + let source = ctx.read_batch(batch).unwrap(); + + let source_name = TableReference::parse_str("source"); + let target_name = TableReference::parse_str("target"); + + let source = LogicalPlanBuilder::scan( + source_name.clone(), + provider_as_source(source.into_view()), + None, + ) + .unwrap() + .build() + .unwrap(); + + let join_predicate = col(Column { + relation: Some(source_name.clone()), + name: "id".to_owned(), + }) + .eq(col(Column { + relation: Some(target_name.clone()), + name: "id".to_owned(), + })); + + let pred = try_construct_early_filter( + join_predicate, + table.snapshot().unwrap(), + &ctx.state(), + &source, + &source_name, + &target_name, + ) + .await + .unwrap(); + + assert!(pred.is_some()); + + let split_pred = { + fn split(expr: Expr, parts: &mut Vec<(String, String)>) { + match expr { + Expr::BinaryExpr(ex) if ex.op == Operator::Or => { + split(*ex.left, parts); + split(*ex.right, parts); + } + Expr::BinaryExpr(ex) if ex.op == Operator::Eq => { + let col = match *ex.right { + Expr::Column(col) => col.name, + ex => panic!("expected column in pred, got {ex}!"), + }; + + let value = match *ex.left { + Expr::Literal(ScalarValue::Utf8(Some(value))) => value, + ex => panic!("expected value in predicate, got {ex}!"), + }; + + parts.push((col, value)) + } + + expr => panic!("expected either = or OR, got {expr}"), + } + } + + let mut parts = vec![]; + split(pred.unwrap(), &mut parts); + parts.sort(); + parts + }; + + let expected_pred_parts = [ + ("id".to_owned(), "B".to_owned()), + ("id".to_owned(), "C".to_owned()), + ("id".to_owned(), "X".to_owned()), + ]; + + assert_eq!(split_pred, expected_pred_parts); + } } diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/core/src/operations/mod.rs similarity index 84% rename from crates/deltalake-core/src/operations/mod.rs rename to crates/core/src/operations/mod.rs index 99a7303691..666b2dc66a 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -15,13 +15,11 @@ use crate::table::builder::DeltaTableBuilder; use crate::DeltaTable; use std::collections::HashMap; -#[cfg(feature = "arrow")] pub mod cast; -#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod convert_to_delta; pub mod create; +pub mod drop_constraints; pub mod filesystem_check; -#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod optimize; pub mod restore; pub mod transaction; @@ -29,17 +27,19 @@ pub mod vacuum; #[cfg(feature = "datafusion")] use self::{ - datafusion_utils::Expression, delete::DeleteBuilder, load::LoadBuilder, merge::MergeBuilder, + constraints::ConstraintBuilder, datafusion_utils::Expression, delete::DeleteBuilder, + drop_constraints::DropConstraintBuilder, load::LoadBuilder, merge::MergeBuilder, update::UpdateBuilder, write::WriteBuilder, }; #[cfg(feature = "datafusion")] pub use ::datafusion::physical_plan::common::collect as collect_sendable_stream; #[cfg(feature = "datafusion")] use arrow::record_batch::RecordBatch; -#[cfg(all(feature = "arrow", feature = "parquet"))] use optimize::OptimizeBuilder; use restore::RestoreBuilder; +#[cfg(feature = "datafusion")] +pub mod constraints; #[cfg(feature = "datafusion")] pub mod delete; #[cfg(feature = "datafusion")] @@ -50,9 +50,10 @@ pub mod merge; pub mod update; #[cfg(feature = "datafusion")] pub mod write; -#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; +// TODO make ops consume a snapshot ... + /// High level interface for executing commands against a DeltaTable pub struct DeltaOps(pub DeltaTable); @@ -130,7 +131,7 @@ impl DeltaOps { #[cfg(feature = "datafusion")] #[must_use] pub fn load(self) -> LoadBuilder { - LoadBuilder::new(self.0.log_store, self.0.state) + LoadBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Write data to Delta table @@ -143,40 +144,39 @@ impl DeltaOps { /// Vacuum stale files from delta table #[must_use] pub fn vacuum(self) -> VacuumBuilder { - VacuumBuilder::new(self.0.log_store, self.0.state) + VacuumBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Audit active files with files present on the filesystem #[must_use] pub fn filesystem_check(self) -> FileSystemCheckBuilder { - FileSystemCheckBuilder::new(self.0.log_store, self.0.state) + FileSystemCheckBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Audit active files with files present on the filesystem - #[cfg(all(feature = "arrow", feature = "parquet"))] #[must_use] pub fn optimize<'a>(self) -> OptimizeBuilder<'a> { - OptimizeBuilder::new(self.0.log_store, self.0.state) + OptimizeBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Delete data from Delta table #[cfg(feature = "datafusion")] #[must_use] pub fn delete(self) -> DeleteBuilder { - DeleteBuilder::new(self.0.log_store, self.0.state) + DeleteBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Update data from Delta table #[cfg(feature = "datafusion")] #[must_use] pub fn update(self) -> UpdateBuilder { - UpdateBuilder::new(self.0.log_store, self.0.state) + UpdateBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Restore delta table to a specified version or datetime #[must_use] pub fn restore(self) -> RestoreBuilder { - RestoreBuilder::new(self.0.log_store, self.0.state) + RestoreBuilder::new(self.0.log_store, self.0.state.unwrap()) } /// Update data from Delta table @@ -187,7 +187,26 @@ impl DeltaOps { source: datafusion::prelude::DataFrame, predicate: E, ) -> MergeBuilder { - MergeBuilder::new(self.0.log_store, self.0.state, predicate.into(), source) + MergeBuilder::new( + self.0.log_store, + self.0.state.unwrap(), + predicate.into(), + source, + ) + } + + /// Add a check constraint to a table + #[cfg(feature = "datafusion")] + #[must_use] + pub fn add_constraint(self) -> ConstraintBuilder { + ConstraintBuilder::new(self.0.log_store, self.0.state.unwrap()) + } + + /// Drops constraints from a table + #[cfg(feature = "datafusion")] + #[must_use] + pub fn drop_constraints(self) -> DropConstraintBuilder { + DropConstraintBuilder::new(self.0.log_store, self.0.state.unwrap()) } } diff --git a/crates/deltalake-core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs similarity index 85% rename from crates/deltalake-core/src/operations/optimize.rs rename to crates/core/src/operations/optimize.rs index 7eb046341f..3ddee7182a 100644 --- a/crates/deltalake-core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -24,25 +24,27 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; +use arrow::datatypes::SchemaRef as ArrowSchemaRef; use arrow_array::RecordBatch; use futures::future::BoxFuture; use futures::stream::BoxStream; use futures::{Future, StreamExt, TryStreamExt}; +use indexmap::IndexMap; use itertools::Itertools; -use log::debug; use num_cpus; use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; use parquet::basic::{Compression, ZstdLevel}; use parquet::errors::ParquetError; use parquet::file::properties::WriterProperties; use serde::{Deserialize, Serialize}; +use tracing::debug; -use super::transaction::{commit, PROTOCOL}; +use super::transaction::PROTOCOL; use super::writer::{PartitionWriter, PartitionWriterConfig}; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Action, Remove}; +use crate::kernel::{Action, PartitionsExt, Remove, Scalar}; use crate::logstore::LogStoreRef; +use crate::operations::transaction::{CommitBuilder, CommitProperties}; use crate::protocol::DeltaOperation; use crate::storage::ObjectStoreRef; use crate::table::state::DeltaTableState; @@ -163,8 +165,8 @@ pub struct OptimizeBuilder<'a> { target_size: Option, /// Properties passed to underlying parquet writer writer_properties: Option, - /// Additional metadata to be added to commit - app_metadata: Option>, + /// Commit properties and configuration + commit_properties: CommitProperties, /// Whether to preserve insertion order within files (default false) preserve_insertion_order: bool, /// Max number of concurrent tasks (default is number of cpus) @@ -185,7 +187,7 @@ impl<'a> OptimizeBuilder<'a> { filters: &[], target_size: None, writer_properties: None, - app_metadata: None, + commit_properties: CommitProperties::default(), preserve_insertion_order: false, max_concurrent_tasks: num_cpus::get(), max_spill_size: 20 * 1024 * 1024 * 2014, // 20 GB. @@ -218,12 +220,9 @@ impl<'a> OptimizeBuilder<'a> { self } - /// Additional metadata to be added to commit info - pub fn with_metadata( - mut self, - metadata: impl IntoIterator, - ) -> Self { - self.app_metadata = Some(HashMap::from_iter(metadata)); + /// Additonal information to write to the commit + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; self } @@ -260,7 +259,7 @@ impl<'a> std::future::IntoFuture for OptimizeBuilder<'a> { let this = self; Box::pin(async move { - PROTOCOL.can_write_to(&this.snapshot)?; + PROTOCOL.can_write_to(&this.snapshot.snapshot)?; let writer_properties = this.writer_properties.unwrap_or_else(|| { WriterProperties::builder() @@ -282,6 +281,7 @@ impl<'a> std::future::IntoFuture for OptimizeBuilder<'a> { this.max_concurrent_tasks, this.max_spill_size, this.min_commit_interval, + this.commit_properties, ) .await?; let mut table = DeltaTable::new_with_state(this.log_store, this.snapshot); @@ -294,20 +294,21 @@ impl<'a> std::future::IntoFuture for OptimizeBuilder<'a> { #[derive(Debug, Clone)] struct OptimizeInput { target_size: i64, + predicate: Option, } impl From for DeltaOperation { fn from(opt_input: OptimizeInput) -> Self { DeltaOperation::Optimize { target_size: opt_input.target_size, - predicate: None, + predicate: opt_input.predicate, } } } fn create_remove( path: &str, - partitions: &HashMap>, + partitions: &IndexMap, size: i64, ) -> Result { // NOTE unwrap is safe since UNIX_EPOCH will always be earlier then now. @@ -319,7 +320,21 @@ fn create_remove( deletion_timestamp: Some(deletion_time), data_change: false, extended_file_metadata: None, - partition_values: Some(partitions.to_owned()), + partition_values: Some( + partitions + .iter() + .map(|(k, v)| { + ( + k.clone(), + if v.is_null() { + None + } else { + Some(v.serialize()) + }, + ) + }) + .collect(), + ), size: Some(size), deletion_vector: None, tags: None, @@ -338,9 +353,12 @@ enum OptimizeOperations { /// /// Bins are determined by the bin-packing algorithm to reach an optimal size. /// Files that are large enough already are skipped. Bins of size 1 are dropped. - Compact(HashMap>), + Compact(HashMap, Vec)>), /// Plan to Z-order each partition - ZOrder(Vec, HashMap), + ZOrder( + Vec, + HashMap, MergeBin)>, + ), // TODO: Sort } @@ -369,8 +387,6 @@ pub struct MergeTaskParameters { input_parameters: OptimizeInput, /// Schema of written files file_schema: ArrowSchemaRef, - /// Column names the table is partitioned by. - partition_columns: Vec, /// Properties passed to parquet writer writer_properties: WriterProperties, } @@ -385,7 +401,7 @@ impl MergePlan { /// collected during the operation. async fn rewrite_files( task_parameters: Arc, - partition: PartitionTuples, + partition_values: IndexMap, files: MergeBin, object_store: ObjectStoreRef, read_stream: F, @@ -393,9 +409,8 @@ impl MergePlan { where F: Future> + Send + 'static, { - debug!("Rewriting files in partition: {:?}", partition); + debug!("Rewriting files in partition: {:?}", partition_values); // First, initialize metrics - let partition_values = partition.to_hashmap(); let mut partial_actions = files .iter() .map(|file_meta| { @@ -429,7 +444,6 @@ impl MergePlan { let writer_config = PartitionWriterConfig::try_new( task_parameters.file_schema.clone(), partition_values.clone(), - task_parameters.partition_columns.clone(), Some(task_parameters.writer_properties.clone()), Some(task_parameters.input_parameters.target_size as usize), None, @@ -441,8 +455,12 @@ impl MergePlan { while let Some(maybe_batch) = read_stream.next().await { let mut batch = maybe_batch?; - batch = - super::cast::cast_record_batch(&batch, task_parameters.file_schema.clone(), false)?; + batch = super::cast::cast_record_batch( + &batch, + task_parameters.file_schema.clone(), + false, + false, + )?; partial_metrics.num_batches += 1; writer.write(&batch).await.map_err(DeltaTableError::from)?; } @@ -462,7 +480,10 @@ impl MergePlan { }); partial_actions.extend(add_actions); - debug!("Finished rewriting files in partition: {:?}", partition); + debug!( + "Finished rewriting files in partition: {:?}", + partition_values + ); Ok((partial_actions, partial_metrics)) } @@ -533,8 +554,9 @@ impl MergePlan { context: Arc, ) -> Result>, DeltaTableError> { use datafusion::prelude::{col, ParquetReadOptions}; - use datafusion_expr::expr::ScalarUDF; - use datafusion_expr::Expr; + use datafusion_common::Column; + use datafusion_expr::expr::ScalarFunction; + use datafusion_expr::{Expr, ScalarUDF}; let locations = files .iter() @@ -549,14 +571,18 @@ impl MergePlan { .schema() .fields() .iter() - .map(|f| col(f.name())) + .map(|f| Expr::Column(Column::from_qualified_name_ignore_case(f.name()))) .collect_vec(); // Add a temporary z-order column we will sort by, and then drop. const ZORDER_KEY_COLUMN: &str = "__zorder_key"; - let cols = context.columns.iter().map(col).collect_vec(); - let expr = Expr::ScalarUDF(ScalarUDF::new( - Arc::new(zorder::datafusion::zorder_key_udf()), + let cols = context + .columns + .iter() + .map(|col| Expr::Column(Column::from_qualified_name_ignore_case(col))) + .collect_vec(); + let expr = Expr::ScalarFunction(ScalarFunction::new_udf( + Arc::new(ScalarUDF::from(zorder::datafusion::ZOrderUDF)), cols, )); let df = df.with_column(ZORDER_KEY_COLUMN, expr)?; @@ -584,12 +610,13 @@ impl MergePlan { #[allow(unused_variables)] // used behind a feature flag max_spill_size: usize, min_commit_interval: Option, + commit_properties: CommitProperties, ) -> Result { let operations = std::mem::take(&mut self.operations); let stream = match operations { OptimizeOperations::Compact(bins) => futures::stream::iter(bins) - .flat_map(|(partition, bins)| { + .flat_map(|(_, (partition, bins))| { futures::stream::iter(bins).map(move |bin| (partition.clone(), bin)) }) .map(|(partition, files)| { @@ -646,7 +673,7 @@ impl MergePlan { let task_parameters = self.task_parameters.clone(); let log_store = log_store.clone(); futures::stream::iter(bins) - .map(move |(partition, files)| { + .map(move |(_, (partition, files))| { let batch_stream = Self::read_zorder(files.clone(), exec_context.clone()); let rewrite_result = tokio::task::spawn(Self::rewrite_files( task_parameters.clone(), @@ -697,28 +724,34 @@ impl MergePlan { last_commit = now; buffered_metrics.preserve_insertion_order = true; - let mut metadata = HashMap::new(); - metadata.insert("readVersion".to_owned(), self.read_table_version.into()); + let mut properties = CommitProperties::default(); + properties.app_metadata = commit_properties.app_metadata.clone(); + properties + .app_metadata + .insert("readVersion".to_owned(), self.read_table_version.into()); let maybe_map_metrics = serde_json::to_value(std::mem::replace( &mut buffered_metrics, orig_metrics.clone(), )); if let Ok(map) = maybe_map_metrics { - metadata.insert("operationMetrics".to_owned(), map); + properties + .app_metadata + .insert("operationMetrics".to_owned(), map); } - table.update_incremental(None).await?; + table.update().await?; debug!("committing {} actions", actions.len()); //// TODO: Check for remove actions on optimized partitions. If a //// optimized partition was updated then abort the commit. Requires (#593). - commit( - table.log_store.as_ref(), - &actions, - self.task_parameters.input_parameters.clone().into(), - table.get_state(), - Some(metadata), - ) - .await?; + + CommitBuilder::from(properties) + .with_actions(actions) + .build( + Some(snapshot), + log_store.clone(), + self.task_parameters.input_parameters.clone().into(), + )? + .await?; } if end { @@ -738,27 +771,6 @@ impl MergePlan { } } -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -struct PartitionTuples(Vec<(String, Option)>); - -impl PartitionTuples { - fn from_hashmap( - partition_columns: &[String], - partition_values: &HashMap>, - ) -> Self { - let mut tuples = Vec::new(); - for column in partition_columns { - let value = partition_values.get(column).cloned().flatten(); - tuples.push((column.clone(), value)); - } - Self(tuples) - } - - fn to_hashmap(&self) -> HashMap> { - self.0.iter().cloned().collect() - } -} - /// Build a Plan on which files to merge together. See [OptimizeBuilder] pub fn create_merge_plan( optimize_type: OptimizeType, @@ -768,33 +780,21 @@ pub fn create_merge_plan( writer_properties: WriterProperties, ) -> Result { let target_size = target_size.unwrap_or_else(|| snapshot.table_config().target_file_size()); - - let partitions_keys = &snapshot - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns; + let partitions_keys = &snapshot.metadata().partition_columns; let (operations, metrics) = match optimize_type { - OptimizeType::Compact => { - build_compaction_plan(snapshot, partitions_keys, filters, target_size)? - } + OptimizeType::Compact => build_compaction_plan(snapshot, filters, target_size)?, OptimizeType::ZOrder(zorder_columns) => { build_zorder_plan(zorder_columns, snapshot, partitions_keys, filters)? } }; - let input_parameters = OptimizeInput { target_size }; - let file_schema = arrow_schema_without_partitions( - &Arc::new( - >::try_from( - &snapshot - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .schema, - )?, - ), - partitions_keys, - ); + let input_parameters = OptimizeInput { + target_size, + predicate: serde_json::to_string(filters).ok(), + }; + let file_schema = + arrow_schema_without_partitions(&Arc::new(snapshot.schema().try_into()?), partitions_keys); Ok(MergePlan { operations, @@ -802,7 +802,6 @@ pub fn create_merge_plan( task_parameters: Arc::new(MergeTaskParameters { input_parameters, file_schema, - partition_columns: partitions_keys.clone(), writer_properties, }), read_table_version: snapshot.version(), @@ -855,33 +854,41 @@ impl IntoIterator for MergeBin { fn build_compaction_plan( snapshot: &DeltaTableState, - partition_keys: &[String], filters: &[PartitionFilter], target_size: i64, ) -> Result<(OptimizeOperations, Metrics), DeltaTableError> { let mut metrics = Metrics::default(); - let mut partition_files: HashMap> = HashMap::new(); + let mut partition_files: HashMap, Vec)> = + HashMap::new(); for add in snapshot.get_active_add_actions_by_partitions(filters)? { + let add = add?; metrics.total_considered_files += 1; - let object_meta = ObjectMeta::try_from(add)?; + let object_meta = ObjectMeta::try_from(&add)?; if (object_meta.size as i64) > target_size { metrics.total_files_skipped += 1; continue; } + let partition_values = add + .partition_values()? + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect::>(); - let part = PartitionTuples::from_hashmap(partition_keys, &add.partition_values); - - partition_files.entry(part).or_default().push(object_meta); + partition_files + .entry(add.partition_values()?.hive_partition_path()) + .or_insert_with(|| (partition_values, vec![])) + .1 + .push(object_meta); } - for file in partition_files.values_mut() { + for (_, file) in partition_files.values_mut() { // Sort files by size: largest to smallest file.sort_by(|a, b| b.size.cmp(&a.size)); } - let mut operations: HashMap> = HashMap::new(); - for (part, files) in partition_files { + let mut operations: HashMap, Vec)> = HashMap::new(); + for (part, (partition, files)) in partition_files { let mut merge_bins = vec![MergeBin::new()]; 'files: for file in files { @@ -898,11 +905,11 @@ fn build_compaction_plan( merge_bins.push(new_bin); } - operations.insert(part, merge_bins); + operations.insert(part, (partition, merge_bins)); } // Prune merge bins with only 1 file, since they have no effect - for (_, bins) in operations.iter_mut() { + for (_, (_, bins)) in operations.iter_mut() { bins.retain(|bin| { if bin.len() == 1 { metrics.total_files_skipped += 1; @@ -912,7 +919,7 @@ fn build_compaction_plan( } }) } - operations.retain(|_, files| !files.is_empty()); + operations.retain(|_, (_, files)| !files.is_empty()); metrics.partitions_optimized = operations.len() as u64; @@ -940,9 +947,7 @@ fn build_zorder_plan( ))); } let field_names = snapshot - .metadata() - .unwrap() - .schema + .schema() .fields() .iter() .map(|field| field.name().to_string()) @@ -960,15 +965,21 @@ fn build_zorder_plan( // For now, just be naive and optimize all files in each selected partition. let mut metrics = Metrics::default(); - let mut partition_files: HashMap = HashMap::new(); + let mut partition_files: HashMap, MergeBin)> = HashMap::new(); for add in snapshot.get_active_add_actions_by_partitions(filters)? { + let add = add?; + let partition_values = add + .partition_values()? + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect::>(); metrics.total_considered_files += 1; - let object_meta = ObjectMeta::try_from(add)?; - let part = PartitionTuples::from_hashmap(partition_keys, &add.partition_values); + let object_meta = ObjectMeta::try_from(&add)?; partition_files - .entry(part) - .or_insert_with(MergeBin::new) + .entry(partition_values.hive_partition_path()) + .or_insert_with(|| (partition_values, MergeBin::new())) + .1 .add(object_meta); } @@ -1073,6 +1084,7 @@ pub(super) mod zorder { use arrow_buffer::bit_util::{get_bit_raw, set_bit_raw, unset_bit_raw}; use arrow_row::{Row, RowConverter, SortField}; use arrow_schema::ArrowError; + // use arrow_schema::Schema as ArrowSchema; /// Execution context for Z-order scan #[cfg(not(feature = "datafusion"))] @@ -1136,8 +1148,11 @@ pub(super) mod zorder { }; use arrow_schema::DataType; use datafusion_common::DataFusionError; - use datafusion_expr::{ColumnarValue, ScalarUDF, Signature, TypeSignature, Volatility}; + use datafusion_expr::{ + ColumnarValue, ScalarUDF, ScalarUDFImpl, Signature, TypeSignature, Volatility, + }; use itertools::Itertools; + use std::any::Any; pub const ZORDER_UDF_NAME: &str = "zorder_key"; @@ -1161,22 +1176,37 @@ pub(super) mod zorder { use url::Url; let ctx = SessionContext::new_with_config_rt(SessionConfig::default(), runtime); - ctx.register_udf(datafusion::zorder_key_udf()); + ctx.register_udf(ScalarUDF::from(datafusion::ZOrderUDF)); Ok(Self { columns, ctx }) } } - /// Get the DataFusion UDF struct for zorder_key - pub fn zorder_key_udf() -> ScalarUDF { - let signature = Signature { - type_signature: TypeSignature::VariadicAny, - volatility: Volatility::Immutable, - }; - ScalarUDF { - name: ZORDER_UDF_NAME.to_string(), - signature, - return_type: Arc::new(|_| Ok(Arc::new(DataType::Binary))), - fun: Arc::new(zorder_key_datafusion), + // DataFusion UDF impl for zorder_key + #[derive(Debug)] + pub struct ZOrderUDF; + + impl ScalarUDFImpl for ZOrderUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + ZORDER_UDF_NAME + } + + fn signature(&self) -> &Signature { + &Signature { + type_signature: TypeSignature::VariadicAny, + volatility: Volatility::Immutable, + } + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Binary) + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + zorder_key_datafusion(args) } } @@ -1208,6 +1238,7 @@ pub(super) mod zorder { use ::datafusion::assert_batches_eq; use arrow_array::{Int32Array, StringArray}; use arrow_ord::sort::sort_to_indices; + use arrow_schema::Field; use arrow_select::take::take; use rand::Rng; #[test] @@ -1300,6 +1331,43 @@ pub(super) mod zorder { } array } + + #[tokio::test] + async fn test_zorder_mixed_case() { + use arrow_schema::Schema as ArrowSchema; + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("moDified", DataType::Utf8, true), + Field::new("ID", DataType::Utf8, true), + Field::new("vaLue", DataType::Int32, true), + ])); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-01", + "2021-02-01", + "2021-02-02", + "2021-02-02", + ])), + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "C", "D"])), + Arc::new(arrow::array::Int32Array::from(vec![1, 10, 20, 100])), + ], + ) + .unwrap(); + // write some data + let table = crate::DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(crate::protocol::SaveMode::Append) + .await + .unwrap(); + + let res = crate::DeltaOps(table) + .optimize() + .with_type(OptimizeType::ZOrder(vec!["moDified".into()])) + .await; + assert!(res.is_ok()); + } } } diff --git a/crates/deltalake-core/src/operations/restore.rs b/crates/core/src/operations/restore.rs similarity index 83% rename from crates/deltalake-core/src/operations/restore.rs rename to crates/core/src/operations/restore.rs index a10248bcb0..a79fb389f1 100644 --- a/crates/deltalake-core/src/operations/restore.rs +++ b/crates/core/src/operations/restore.rs @@ -33,11 +33,12 @@ use serde::Serialize; use crate::kernel::{Action, Add, Protocol, Remove}; use crate::logstore::LogStoreRef; -use crate::operations::transaction::{prepare_commit, TransactionError}; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::{DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; +use super::transaction::{CommitBuilder, CommitProperties, TransactionError}; + /// Errors that can occur during restore #[derive(thiserror::Error, Debug)] enum RestoreError { @@ -84,6 +85,8 @@ pub struct RestoreBuilder { ignore_missing_files: bool, /// Protocol downgrade allowed protocol_downgrade_allowed: bool, + /// Additional information to add to the commit + commit_properties: CommitProperties, } impl RestoreBuilder { @@ -96,6 +99,7 @@ impl RestoreBuilder { datetime_to_restore: None, ignore_missing_files: false, protocol_downgrade_allowed: false, + commit_properties: CommitProperties::default(), } } @@ -123,6 +127,12 @@ impl RestoreBuilder { self.protocol_downgrade_allowed = protocol_downgrade_allowed; self } + + /// Additional metadata to be added to commit info + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + self + } } async fn execute( @@ -132,6 +142,7 @@ async fn execute( datetime_to_restore: Option>, ignore_missing_files: bool, protocol_downgrade_allowed: bool, + mut commit_properties: CommitProperties, ) -> DeltaResult { if !(version_to_restore .is_none() @@ -140,6 +151,7 @@ async fn execute( return Err(DeltaTableError::from(RestoreError::InvalidRestoreParameter)); } let mut table = DeltaTable::new(log_store.clone(), DeltaTableConfig::default()); + let version = match datetime_to_restore { Some(datetime) => { table.load_with_datetime(datetime).await?; @@ -157,8 +169,8 @@ async fn execute( snapshot.version(), ))); } - let state_to_restore_files = table.get_state().files().clone(); - let latest_state_files = snapshot.files().clone(); + let state_to_restore_files = table.snapshot()?.file_actions()?; + let latest_state_files = snapshot.file_actions()?; let state_to_restore_files_set = HashSet::::from_iter(state_to_restore_files.iter().cloned()); let latest_state_files_set = HashSet::::from_iter(latest_state_files.iter().cloned()); @@ -207,59 +219,68 @@ async fn execute( let mut actions = vec![]; let protocol = if protocol_downgrade_allowed { Protocol { - min_reader_version: table.protocol().min_reader_version, - min_writer_version: table.protocol().min_writer_version, + min_reader_version: table.protocol()?.min_reader_version, + min_writer_version: table.protocol()?.min_writer_version, writer_features: if snapshot.protocol().min_writer_version < 7 { None } else { - table.protocol().writer_features.clone() + table.protocol()?.writer_features.clone() }, reader_features: if snapshot.protocol().min_reader_version < 3 { None } else { - table.protocol().reader_features.clone() + table.protocol()?.reader_features.clone() }, } } else { Protocol { min_reader_version: max( - table.protocol().min_reader_version, + table.protocol()?.min_reader_version, snapshot.protocol().min_reader_version, ), min_writer_version: max( - table.protocol().min_writer_version, + table.protocol()?.min_writer_version, snapshot.protocol().min_writer_version, ), writer_features: snapshot.protocol().writer_features.clone(), reader_features: snapshot.protocol().reader_features.clone(), } }; + commit_properties + .app_metadata + .insert("readVersion".to_owned(), snapshot.version().into()); + commit_properties.app_metadata.insert( + "operationMetrics".to_owned(), + serde_json::to_value(&metrics)?, + ); + actions.push(Action::Protocol(protocol)); actions.extend(files_to_add.into_iter().map(Action::Add)); actions.extend(files_to_remove.into_iter().map(Action::Remove)); - let commit = prepare_commit( - log_store.object_store().as_ref(), - &DeltaOperation::Restore { - version: version_to_restore, - datetime: datetime_to_restore.map(|time| -> i64 { time.timestamp_millis() }), - }, - &actions, - None, - ) - .await?; + let operation = DeltaOperation::Restore { + version: version_to_restore, + datetime: datetime_to_restore.map(|time| -> i64 { time.timestamp_millis() }), + }; + + let prepared_commit = CommitBuilder::from(commit_properties) + .with_actions(actions) + .build(Some(&snapshot), log_store.clone(), operation)? + .into_prepared_commit_future() + .await?; + let commit_version = snapshot.version() + 1; - match log_store.write_commit_entry(commit_version, &commit).await { + let commit = prepared_commit.path(); + match log_store.write_commit_entry(commit_version, commit).await { Ok(_) => {} Err(err @ TransactionError::VersionAlreadyExists(_)) => { return Err(err.into()); } Err(err) => { - log_store.object_store().delete(&commit).await?; + log_store.object_store().delete(commit).await?; return Err(err.into()); } } - Ok(metrics) } @@ -297,6 +318,7 @@ impl std::future::IntoFuture for RestoreBuilder { this.datetime_to_restore, this.ignore_missing_files, this.protocol_downgrade_allowed, + this.commit_properties, ) .await?; let mut table = DeltaTable::new_with_state(this.log_store, this.snapshot); diff --git a/crates/deltalake-core/src/operations/transaction/conflict_checker.rs b/crates/core/src/operations/transaction/conflict_checker.rs similarity index 96% rename from crates/deltalake-core/src/operations/transaction/conflict_checker.rs rename to crates/core/src/operations/transaction/conflict_checker.rs index eed34ba595..45da0007cb 100644 --- a/crates/deltalake-core/src/operations/transaction/conflict_checker.rs +++ b/crates/core/src/operations/transaction/conflict_checker.rs @@ -2,12 +2,13 @@ use std::collections::HashSet; use super::CommitInfo; +use crate::delta_datafusion::DataFusionMixins; use crate::errors::DeltaResult; +use crate::kernel::EagerSnapshot; use crate::kernel::{Action, Add, Metadata, Protocol, Remove}; use crate::logstore::{get_actions, LogStore}; use crate::protocol::DeltaOperation; use crate::table::config::IsolationLevel; -use crate::table::state::DeltaTableState; use crate::DeltaTableError; #[cfg(feature = "datafusion")] @@ -98,9 +99,9 @@ pub(crate) struct TransactionInfo<'a> { /// appIds that have been seen by the transaction pub(crate) read_app_ids: HashSet, /// delta log actions that the transaction wants to commit - actions: &'a Vec, + actions: &'a [Action], /// read [`DeltaTableState`] used for the transaction - pub(crate) read_snapshot: &'a DeltaTableState, + pub(crate) read_snapshot: &'a EagerSnapshot, /// Whether the transaction tainted the whole table read_whole_table: bool, } @@ -108,9 +109,9 @@ pub(crate) struct TransactionInfo<'a> { impl<'a> TransactionInfo<'a> { #[cfg(feature = "datafusion")] pub fn try_new( - read_snapshot: &'a DeltaTableState, + read_snapshot: &'a EagerSnapshot, read_predicates: Option, - actions: &'a Vec, + actions: &'a [Action], read_whole_table: bool, ) -> DeltaResult { use datafusion::prelude::SessionContext; @@ -132,7 +133,7 @@ impl<'a> TransactionInfo<'a> { #[cfg(feature = "datafusion")] #[allow(unused)] pub fn new( - read_snapshot: &'a DeltaTableState, + read_snapshot: &'a EagerSnapshot, read_predicates: Option, actions: &'a Vec, read_whole_table: bool, @@ -149,7 +150,7 @@ impl<'a> TransactionInfo<'a> { #[cfg(not(feature = "datafusion"))] pub fn try_new( - read_snapshot: &'a DeltaTableState, + read_snapshot: &'a EagerSnapshot, read_predicates: Option, actions: &'a Vec, read_whole_table: bool, @@ -173,14 +174,16 @@ impl<'a> TransactionInfo<'a> { #[cfg(feature = "datafusion")] /// Files read by the transaction - pub fn read_files(&self) -> Result, CommitConflictError> { + pub fn read_files(&self) -> Result + '_, CommitConflictError> { + use crate::delta_datafusion::files_matching_predicate; + if let Some(predicate) = &self.read_predicates { Ok(Either::Left( - self.read_snapshot - .files_matching_predicate(&[predicate.clone()]) - .map_err(|err| CommitConflictError::Predicate { + files_matching_predicate(self.read_snapshot, &[predicate.clone()]).map_err( + |err| CommitConflictError::Predicate { source: Box::new(err), - })?, + }, + )?, )) } else { Ok(Either::Right(std::iter::empty())) @@ -189,8 +192,8 @@ impl<'a> TransactionInfo<'a> { #[cfg(not(feature = "datafusion"))] /// Files read by the transaction - pub fn read_files(&self) -> Result, CommitConflictError> { - Ok(self.read_snapshot.files().iter()) + pub fn read_files(&self) -> Result, CommitConflictError> { + Ok(self.read_snapshot.file_actions().unwrap().into_iter()) } /// Whether the whole table was read during the transaction @@ -463,7 +466,6 @@ impl<'a> ConflictChecker<'a> { .txn_info .read_snapshot .metadata() - .ok_or(CommitConflictError::NoMetadata)? .partition_columns; AddContainer::new(&added_files_to_check, partition_columns, arrow_schema) .predicate_matches(predicate.clone()) @@ -678,9 +680,12 @@ mod tests { actions: Vec, read_whole_table: bool, ) -> Result<(), CommitConflictError> { + use crate::table::state::DeltaTableState; + let setup_actions = setup.unwrap_or_else(|| init_table_actions(None)); - let state = DeltaTableState::from_actions(setup_actions, 0).unwrap(); - let transaction_info = TransactionInfo::new(&state, reads, &actions, read_whole_table); + let state = DeltaTableState::from_actions(setup_actions).unwrap(); + let snapshot = state.snapshot(); + let transaction_info = TransactionInfo::new(snapshot, reads, &actions, read_whole_table); let summary = WinningCommitSummary { actions: concurrent, commit_info: None, diff --git a/crates/core/src/operations/transaction/mod.rs b/crates/core/src/operations/transaction/mod.rs new file mode 100644 index 0000000000..64cb25c0be --- /dev/null +++ b/crates/core/src/operations/transaction/mod.rs @@ -0,0 +1,605 @@ +//! Add a commit entry to the Delta Table. +//! This module provides a unified interface for modifying commit behavior and attributes +//! +//! [`CommitProperties`] provides an unified client interface for +//! all Delta opeartions. Internally this is used to initialize a [`CommitBuilder`] +//! +//! For advanced use cases [`CommitBuilder`] can be used which allows +//! finer control over the commit process. The builder can be converted +//! into a future the yield either a [`PreparedCommit`] or a [`FinalizedCommit`]. +//! +//! A [`PreparedCommit`] represents a temporary commit marker written to storage. +//! To convert to a [`FinalizedCommit`] an atomic rename is attempted. If the rename fails +//! then conflict resolution is performed and the atomic rename is tried for the latest version. +//! +//!

+//!                                          Client Interface
+//!        ┌─────────────────────────────┐                    
+//!        │      Commit Properties      │                    
+//!        │                             │                    
+//!        │ Public commit interface for │                    
+//!        │     all Delta Operations    │                    
+//!        │                             │                    
+//!        └─────────────┬───────────────┘                    
+//!                      │                                    
+//! ─────────────────────┼────────────────────────────────────
+//!                      │                                    
+//!                      ▼                  Advanced Interface
+//!        ┌─────────────────────────────┐                    
+//!        │       Commit Builder        │                    
+//!        │                             │                    
+//!        │   Advanced entry point for  │                    
+//!        │     creating a commit       │                    
+//!        └─────────────┬───────────────┘                    
+//!                      │                                    
+//!                      ▼                                    
+//!     ┌───────────────────────────────────┐                 
+//!     │                                   │                 
+//!     │ ┌───────────────────────────────┐ │                 
+//!     │ │        Prepared Commit        │ │                 
+//!     │ │                               │ │                 
+//!     │ │     Represents a temporary    │ │                 
+//!     │ │   commit marker written to    │ │                 
+//!     │ │           storage             │ │                 
+//!     │ └──────────────┬────────────────┘ │                 
+//!     │                │                  │                 
+//!     │                ▼                  │                 
+//!     │ ┌───────────────────────────────┐ │                 
+//!     │ │       Finalize Commit         │ │                 
+//!     │ │                               │ │                 
+//!     │ │   Convert the commit marker   │ │                 
+//!     │ │   to a commit using atomic    │ │                 
+//!     │ │         operations            │ │                 
+//!     │ │                               │ │                 
+//!     │ └───────────────────────────────┘ │                 
+//!     │                                   │                 
+//!     └────────────────┬──────────────────┘                 
+//!                      │                                    
+//!                      ▼                                    
+//!       ┌───────────────────────────────┐                   
+//!       │        Finalized Commit       │                   
+//!       │                               │                   
+//!       │ Commit that was materialized  │                   
+//!       │         to storage            │                   
+//!       │                               │                   
+//!       └───────────────────────────────┘                   
+//!
+ +use std::collections::HashMap; + +use chrono::Utc; +use conflict_checker::ConflictChecker; +use futures::future::BoxFuture; +use object_store::path::Path; +use object_store::{Error as ObjectStoreError, ObjectStore}; +use serde_json::Value; + +use self::conflict_checker::{CommitConflictError, TransactionInfo, WinningCommitSummary}; +use crate::errors::DeltaTableError; +use crate::kernel::{ + Action, CommitInfo, EagerSnapshot, Metadata, Protocol, ReaderFeatures, WriterFeatures, +}; +use crate::logstore::LogStoreRef; +use crate::protocol::DeltaOperation; +use crate::storage::ObjectStoreRetryExt; +use crate::table::config::TableConfig; +use crate::table::state::DeltaTableState; +use crate::{crate_version, DeltaResult}; + +pub use self::protocol::INSTANCE as PROTOCOL; + +mod conflict_checker; +mod protocol; +#[cfg(feature = "datafusion")] +mod state; +#[cfg(test)] +pub(crate) mod test_utils; + +const DELTA_LOG_FOLDER: &str = "_delta_log"; +const DEFAULT_RETRIES: usize = 15; + +/// Error raised while commititng transaction +#[derive(thiserror::Error, Debug)] +pub enum TransactionError { + /// Version already exists + #[error("Tried committing existing table version: {0}")] + VersionAlreadyExists(i64), + + /// Error returned when reading the delta log object failed. + #[error("Error serializing commit log to json: {json_err}")] + SerializeLogJson { + /// Commit log record JSON serialization error. + json_err: serde_json::error::Error, + }, + + /// Error returned when reading the delta log object failed. + #[error("Log storage error: {}", .source)] + ObjectStore { + /// Storage error details when reading the delta log object failed. + #[from] + source: ObjectStoreError, + }, + + /// Error returned when a commit conflict ocurred + #[error("Failed to commit transaction: {0}")] + CommitConflict(#[from] CommitConflictError), + + /// Error returned when maximum number of commit trioals is exceeded + #[error("Failed to commit transaction: {0}")] + MaxCommitAttempts(i32), + + /// The transaction includes Remove action with data change but Delta table is append-only + #[error( + "The transaction includes Remove action with data change but Delta table is append-only" + )] + DeltaTableAppendOnly, + + /// Error returned when unsupported reader features are required + #[error("Unsupported reader features required: {0:?}")] + UnsupportedReaderFeatures(Vec), + + /// Error returned when unsupported writer features are required + #[error("Unsupported writer features required: {0:?}")] + UnsupportedWriterFeatures(Vec), + + /// Error returned when writer features are required but not specified + #[error("Writer features must be specified for writerversion >= 7, please specify: {0:?}")] + WriterFeaturesRequired(WriterFeatures), + + /// Error returned when reader features are required but not specified + #[error("Reader features must be specified for reader version >= 3, please specify: {0:?}")] + ReaderFeaturesRequired(ReaderFeatures), + + /// The transaction failed to commit due to an error in an implementation-specific layer. + /// Currently used by DynamoDb-backed S3 log store when database operations fail. + #[error("Transaction failed: {msg}")] + LogStoreError { + /// Detailed message for the commit failure. + msg: String, + /// underlying error in the log store transactional layer. + source: Box, + }, +} + +impl From for DeltaTableError { + fn from(err: TransactionError) -> Self { + match err { + TransactionError::VersionAlreadyExists(version) => { + DeltaTableError::VersionAlreadyExists(version) + } + TransactionError::SerializeLogJson { json_err } => { + DeltaTableError::SerializeLogJson { json_err } + } + TransactionError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, + other => DeltaTableError::Transaction { source: other }, + } + } +} + +/// Error raised while commititng transaction +#[derive(thiserror::Error, Debug)] +pub enum CommitBuilderError {} + +impl From for DeltaTableError { + fn from(err: CommitBuilderError) -> Self { + DeltaTableError::CommitValidation { source: err } + } +} + +/// Reference to some structure that contains mandatory attributes for performing a commit. +pub trait TableReference: Send + Sync { + /// Well known table configuration + fn config(&self) -> TableConfig; + + /// Get the table protocol of the snapshot + fn protocol(&self) -> &Protocol; + + /// Get the table metadata of the snapshot + fn metadata(&self) -> &Metadata; + + /// Try to cast this table reference to a `EagerSnapshot` + fn eager_snapshot(&self) -> Option<&EagerSnapshot>; +} + +impl TableReference for EagerSnapshot { + fn protocol(&self) -> &Protocol { + EagerSnapshot::protocol(self) + } + + fn metadata(&self) -> &Metadata { + EagerSnapshot::metadata(self) + } + + fn config(&self) -> TableConfig { + self.table_config() + } + + fn eager_snapshot(&self) -> Option<&EagerSnapshot> { + Some(self) + } +} + +impl TableReference for DeltaTableState { + fn config(&self) -> TableConfig { + self.snapshot.config() + } + + fn protocol(&self) -> &Protocol { + self.snapshot.protocol() + } + + fn metadata(&self) -> &Metadata { + self.snapshot.metadata() + } + + fn eager_snapshot(&self) -> Option<&EagerSnapshot> { + Some(&self.snapshot) + } +} + +/// Data that was actually written to the log store. +pub struct CommitData { + /// The actions + pub actions: Vec, + /// The Operation + pub operation: DeltaOperation, + /// The Metadata + pub app_metadata: HashMap, +} + +impl CommitData { + /// Create new data to be comitted + pub fn new( + mut actions: Vec, + operation: DeltaOperation, + mut app_metadata: HashMap, + ) -> Result { + if !actions.iter().any(|a| matches!(a, Action::CommitInfo(..))) { + let mut commit_info = operation.get_commit_info(); + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + app_metadata.insert( + "clientVersion".to_string(), + Value::String(format!("delta-rs.{}", crate_version())), + ); + app_metadata.extend(commit_info.info); + commit_info.info = app_metadata.clone(); + actions.push(Action::CommitInfo(commit_info)) + } + Ok(CommitData { + actions, + operation, + app_metadata, + }) + } + + /// Convert actions to their json representation + pub fn log_entry_from_actions<'a>( + actions: impl IntoIterator, + ) -> Result { + let mut jsons = Vec::::new(); + for action in actions { + let json = serde_json::to_string(action) + .map_err(|e| TransactionError::SerializeLogJson { json_err: e })?; + jsons.push(json); + } + Ok(jsons.join("\n")) + } + + /// Obtain the byte representation of the commit. + pub fn get_bytes(&self) -> Result { + // Data MUST be read from the passed `CommitData`. Don't add data that is not sourced from there. + let actions = &self.actions; + Ok(bytes::Bytes::from(Self::log_entry_from_actions(actions)?)) + } +} + +#[derive(Clone, Debug)] +/// End user facing interface to be used by operations on the table. +/// Enable controling commit behaviour and modifying metadata that is written during a commit. +pub struct CommitProperties { + pub(crate) app_metadata: HashMap, + max_retries: usize, +} + +impl Default for CommitProperties { + fn default() -> Self { + Self { + app_metadata: Default::default(), + max_retries: DEFAULT_RETRIES, + } + } +} + +impl CommitProperties { + /// Specify metadata the be comitted + pub fn with_metadata( + mut self, + metadata: impl IntoIterator, + ) -> Self { + self.app_metadata = HashMap::from_iter(metadata); + self + } +} + +impl From for CommitBuilder { + fn from(value: CommitProperties) -> Self { + CommitBuilder { + max_retries: value.max_retries, + app_metadata: value.app_metadata, + ..Default::default() + } + } +} + +/// Prepare data to be committed to the Delta log and control how the commit is performed +pub struct CommitBuilder { + actions: Vec, + app_metadata: HashMap, + max_retries: usize, +} + +impl Default for CommitBuilder { + fn default() -> Self { + CommitBuilder { + actions: Vec::new(), + app_metadata: HashMap::new(), + max_retries: DEFAULT_RETRIES, + } + } +} + +impl<'a> CommitBuilder { + /// Actions to be included in the commit + pub fn with_actions(mut self, actions: Vec) -> Self { + self.actions = actions; + self + } + + /// Metadata for the operation performed like metrics, user, and notebook + pub fn with_app_metadata(mut self, app_metadata: HashMap) -> Self { + self.app_metadata = app_metadata; + self + } + + /// Prepare a Commit operation using the configured builder + pub fn build( + self, + table_data: Option<&'a dyn TableReference>, + log_store: LogStoreRef, + operation: DeltaOperation, + ) -> Result, CommitBuilderError> { + let data = CommitData::new(self.actions, operation, self.app_metadata)?; + Ok(PreCommit { + log_store, + table_data, + max_retries: self.max_retries, + data, + }) + } +} + +/// Represents a commit that has not yet started but all details are finalized +pub struct PreCommit<'a> { + log_store: LogStoreRef, + table_data: Option<&'a dyn TableReference>, + data: CommitData, + max_retries: usize, +} + +impl<'a> std::future::IntoFuture for PreCommit<'a> { + type Output = DeltaResult; + type IntoFuture = BoxFuture<'a, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let this = self; + + Box::pin(async move { this.into_prepared_commit_future().await?.await }) + } +} + +impl<'a> PreCommit<'a> { + /// Prepare the commit but do not finalize it + pub fn into_prepared_commit_future(self) -> BoxFuture<'a, DeltaResult>> { + let this = self; + + Box::pin(async move { + if let Some(table_reference) = this.table_data { + PROTOCOL.can_commit(table_reference, &this.data.actions, &this.data.operation)?; + } + + // Serialize all actions that are part of this log entry. + let log_entry = this.data.get_bytes()?; + + // Write delta log entry as temporary file to storage. For the actual commit, + // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. + let token = uuid::Uuid::new_v4().to_string(); + let file_name = format!("_commit_{token}.json.tmp"); + let path = Path::from_iter([DELTA_LOG_FOLDER, &file_name]); + this.log_store.object_store().put(&path, log_entry).await?; + + Ok(PreparedCommit { + path, + log_store: this.log_store, + table_data: this.table_data, + max_retries: this.max_retries, + data: this.data, + }) + }) + } +} + +/// Represents a inflight commit with a temporary commit marker on the log store +pub struct PreparedCommit<'a> { + path: Path, + log_store: LogStoreRef, + data: CommitData, + table_data: Option<&'a dyn TableReference>, + max_retries: usize, +} + +impl<'a> PreparedCommit<'a> { + /// The temporary commit file created + pub fn path(&self) -> &Path { + &self.path + } +} + +impl<'a> std::future::IntoFuture for PreparedCommit<'a> { + type Output = DeltaResult; + type IntoFuture = BoxFuture<'a, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let this = self; + + Box::pin(async move { + let tmp_commit = &this.path; + + if this.table_data.is_none() { + this.log_store.write_commit_entry(0, tmp_commit).await?; + return Ok(FinalizedCommit { + version: 0, + data: this.data, + }); + } + + // unwrap() is safe here due to the above check + // TODO: refactor to only depend on TableReference Trait + let read_snapshot = + this.table_data + .unwrap() + .eager_snapshot() + .ok_or(DeltaTableError::Generic( + "Expected an instance of EagerSnapshot".to_owned(), + ))?; + + let mut attempt_number = 1; + while attempt_number <= this.max_retries { + let version = read_snapshot.version() + attempt_number as i64; + match this.log_store.write_commit_entry(version, tmp_commit).await { + Ok(()) => { + return Ok(FinalizedCommit { + version, + data: this.data, + }) + } + Err(TransactionError::VersionAlreadyExists(version)) => { + let summary = WinningCommitSummary::try_new( + this.log_store.as_ref(), + version - 1, + version, + ) + .await?; + let transaction_info = TransactionInfo::try_new( + read_snapshot, + this.data.operation.read_predicate(), + &this.data.actions, + // TODO allow tainting whole table + false, + )?; + let conflict_checker = ConflictChecker::new( + transaction_info, + summary, + Some(&this.data.operation), + ); + match conflict_checker.check_conflicts() { + Ok(_) => { + attempt_number += 1; + } + Err(err) => { + this.log_store + .object_store() + .delete_with_retries(&tmp_commit, 15) + .await?; + return Err(TransactionError::CommitConflict(err).into()); + } + }; + } + Err(err) => { + this.log_store + .object_store() + .delete_with_retries(&tmp_commit, 15) + .await?; + return Err(err.into()); + } + } + } + + Err(TransactionError::MaxCommitAttempts(this.max_retries as i32).into()) + }) + } +} + +/// A commit that successfully completed +pub struct FinalizedCommit { + /// The winning version number of the commit + pub version: i64, + /// The data that was comitted to the log store + pub data: CommitData, +} + +impl FinalizedCommit { + /// The materialized version of the commit + pub fn version(&self) -> i64 { + self.version + } + + /// Data used to write the commit + pub fn data(&self) -> &CommitData { + &self.data + } +} + +#[cfg(test)] +mod tests { + use std::{collections::HashMap, sync::Arc}; + + use self::test_utils::init_table_actions; + use super::*; + use crate::{ + logstore::{default_logstore::DefaultLogStore, LogStore}, + storage::commit_uri_from_version, + }; + use object_store::memory::InMemory; + use url::Url; + + #[test] + fn test_commit_uri_from_version() { + let version = commit_uri_from_version(0); + assert_eq!(version, Path::from("_delta_log/00000000000000000000.json")); + let version = commit_uri_from_version(123); + assert_eq!(version, Path::from("_delta_log/00000000000000000123.json")) + } + + #[test] + fn test_log_entry_from_actions() { + let actions = init_table_actions(None); + let entry = CommitData::log_entry_from_actions(&actions).unwrap(); + let lines: Vec<_> = entry.lines().collect(); + // writes every action to a line + assert_eq!(actions.len(), lines.len()) + } + + #[tokio::test] + async fn test_try_commit_transaction() { + let store = Arc::new(InMemory::new()); + let url = Url::parse("mem://what/is/this").unwrap(); + let log_store = DefaultLogStore::new( + store.clone(), + crate::logstore::LogStoreConfig { + location: url, + options: HashMap::new().into(), + }, + ); + let tmp_path = Path::from("_delta_log/tmp"); + let version_path = Path::from("_delta_log/00000000000000000000.json"); + store.put(&tmp_path, bytes::Bytes::new()).await.unwrap(); + store.put(&version_path, bytes::Bytes::new()).await.unwrap(); + + let res = log_store.write_commit_entry(0, &tmp_path).await; + // fails if file version already exists + assert!(res.is_err()); + + // succeeds for next version + log_store.write_commit_entry(1, &tmp_path).await.unwrap(); + } +} diff --git a/crates/core/src/operations/transaction/protocol.rs b/crates/core/src/operations/transaction/protocol.rs new file mode 100644 index 0000000000..7971ac883a --- /dev/null +++ b/crates/core/src/operations/transaction/protocol.rs @@ -0,0 +1,542 @@ +use std::collections::HashSet; + +use lazy_static::lazy_static; +use once_cell::sync::Lazy; + +use super::{TableReference, TransactionError}; +use crate::kernel::{Action, DataType, EagerSnapshot, ReaderFeatures, Schema, WriterFeatures}; +use crate::protocol::DeltaOperation; +use crate::table::state::DeltaTableState; + +lazy_static! { + static ref READER_V2: HashSet = + HashSet::from_iter([ReaderFeatures::ColumnMapping]); + static ref WRITER_V2: HashSet = + HashSet::from_iter([WriterFeatures::AppendOnly, WriterFeatures::Invariants]); + static ref WRITER_V3: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints + ]); + static ref WRITER_V4: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints, + WriterFeatures::ChangeDataFeed, + WriterFeatures::GeneratedColumns + ]); + static ref WRITER_V5: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints, + WriterFeatures::ChangeDataFeed, + WriterFeatures::GeneratedColumns, + WriterFeatures::ColumnMapping, + ]); + static ref WRITER_V6: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints, + WriterFeatures::ChangeDataFeed, + WriterFeatures::GeneratedColumns, + WriterFeatures::ColumnMapping, + WriterFeatures::IdentityColumns, + ]); +} + +pub struct ProtocolChecker { + reader_features: HashSet, + writer_features: HashSet, +} + +impl ProtocolChecker { + /// Create a new protocol checker. + pub fn new( + reader_features: HashSet, + writer_features: HashSet, + ) -> Self { + Self { + reader_features, + writer_features, + } + } + + pub fn default_reader_version(&self) -> i32 { + 1 + } + + pub fn default_writer_version(&self) -> i32 { + 2 + } + + /// Check append-only at the high level (operation level) + pub fn check_append_only(&self, snapshot: &EagerSnapshot) -> Result<(), TransactionError> { + if snapshot.table_config().append_only() { + return Err(TransactionError::DeltaTableAppendOnly); + } + Ok(()) + } + + /// Check can write_timestamp_ntz + pub fn check_can_write_timestamp_ntz( + &self, + snapshot: &DeltaTableState, + schema: &Schema, + ) -> Result<(), TransactionError> { + let contains_timestampntz = schema + .fields() + .iter() + .any(|f| f.data_type() == &DataType::TIMESTAMPNTZ); + + let required_features: Option<&HashSet> = + match snapshot.protocol().min_writer_version { + 0..=6 => None, + _ => snapshot.protocol().writer_features.as_ref(), + }; + + if let Some(table_features) = required_features { + if !table_features.contains(&WriterFeatures::TimestampWithoutTimezone) + && contains_timestampntz + { + return Err(TransactionError::WriterFeaturesRequired( + WriterFeatures::TimestampWithoutTimezone, + )); + } + } else if contains_timestampntz { + return Err(TransactionError::WriterFeaturesRequired( + WriterFeatures::TimestampWithoutTimezone, + )); + } + Ok(()) + } + + /// Check if delta-rs can read form the given delta table. + pub fn can_read_from(&self, snapshot: &dyn TableReference) -> Result<(), TransactionError> { + let required_features: Option<&HashSet> = + match snapshot.protocol().min_reader_version { + 0 | 1 => None, + 2 => Some(&READER_V2), + _ => snapshot.protocol().reader_features.as_ref(), + }; + if let Some(features) = required_features { + let mut diff = features.difference(&self.reader_features).peekable(); + if diff.peek().is_some() { + return Err(TransactionError::UnsupportedReaderFeatures( + diff.cloned().collect(), + )); + } + }; + Ok(()) + } + + /// Check if delta-rs can write to the given delta table. + pub fn can_write_to(&self, snapshot: &dyn TableReference) -> Result<(), TransactionError> { + // NOTE: writers must always support all required reader features + self.can_read_from(snapshot)?; + + let required_features: Option<&HashSet> = + match snapshot.protocol().min_writer_version { + 0 | 1 => None, + 2 => Some(&WRITER_V2), + 3 => Some(&WRITER_V3), + 4 => Some(&WRITER_V4), + 5 => Some(&WRITER_V5), + 6 => Some(&WRITER_V6), + _ => snapshot.protocol().writer_features.as_ref(), + }; + + if let Some(features) = required_features { + let mut diff = features.difference(&self.writer_features).peekable(); + if diff.peek().is_some() { + return Err(TransactionError::UnsupportedWriterFeatures( + diff.cloned().collect(), + )); + } + }; + Ok(()) + } + + pub fn can_commit( + &self, + snapshot: &dyn TableReference, + actions: &[Action], + operation: &DeltaOperation, + ) -> Result<(), TransactionError> { + self.can_write_to(snapshot)?; + + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#append-only-tables + let append_only_enabled = if snapshot.protocol().min_writer_version < 2 { + false + } else if snapshot.protocol().min_writer_version < 7 { + snapshot.config().append_only() + } else { + snapshot + .protocol() + .writer_features + .as_ref() + .ok_or(TransactionError::WriterFeaturesRequired( + WriterFeatures::AppendOnly, + ))? + .contains(&WriterFeatures::AppendOnly) + && snapshot.config().append_only() + }; + if append_only_enabled { + match operation { + DeltaOperation::Restore { .. } | DeltaOperation::FileSystemCheck { .. } => {} + _ => { + actions.iter().try_for_each(|action| match action { + Action::Remove(remove) if remove.data_change => { + Err(TransactionError::DeltaTableAppendOnly) + } + _ => Ok(()), + })?; + } + } + } + + Ok(()) + } +} + +/// The global protocol checker instance to validate table versions and features. +/// +/// This instance is used by default in all transaction operations, since feature +/// support is not configurable but rather decided at compile time. +/// +/// As we implement new features, we need to update this instance accordingly. +/// resulting version support is determined by the supported table feature set. +pub static INSTANCE: Lazy = Lazy::new(|| { + let mut reader_features = HashSet::new(); + reader_features.insert(ReaderFeatures::TimestampWithoutTimezone); + // reader_features.insert(ReaderFeatures::ColumnMapping); + + let mut writer_features = HashSet::new(); + writer_features.insert(WriterFeatures::AppendOnly); + writer_features.insert(WriterFeatures::TimestampWithoutTimezone); + #[cfg(feature = "datafusion")] + { + writer_features.insert(WriterFeatures::Invariants); + writer_features.insert(WriterFeatures::CheckConstraints); + } + // writer_features.insert(WriterFeatures::ChangeDataFeed); + // writer_features.insert(WriterFeatures::GeneratedColumns); + // writer_features.insert(WriterFeatures::ColumnMapping); + // writer_features.insert(WriterFeatures::IdentityColumns); + + ProtocolChecker::new(reader_features, writer_features) +}); + +#[cfg(test)] +mod tests { + use super::super::test_utils::create_metadata_action; + use super::*; + use crate::kernel::{Action, Add, Protocol, Remove}; + use crate::protocol::SaveMode; + use crate::table::state::DeltaTableState; + use crate::DeltaConfigKey; + use std::collections::HashMap; + + #[test] + fn test_can_commit_append_only() { + let append_actions = vec![Action::Add(Add { + path: "test".to_string(), + data_change: true, + ..Default::default() + })]; + let append_op = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + + let change_actions = vec![ + Action::Add(Add { + path: "test".to_string(), + data_change: true, + ..Default::default() + }), + Action::Remove(Remove { + path: "test".to_string(), + data_change: true, + ..Default::default() + }), + ]; + let change_op = DeltaOperation::Update { predicate: None }; + + let neutral_actions = vec![ + Action::Add(Add { + path: "test".to_string(), + data_change: false, + ..Default::default() + }), + Action::Remove(Remove { + path: "test".to_string(), + data_change: false, + ..Default::default() + }), + ]; + let neutral_op = DeltaOperation::Update { predicate: None }; + + let create_actions = |writer: i32, append: &str, feat: Vec| { + vec![ + Action::Protocol(Protocol { + min_reader_version: 1, + min_writer_version: writer, + writer_features: Some(feat.into_iter().collect()), + ..Default::default() + }), + create_metadata_action( + None, + Some(HashMap::from([( + DeltaConfigKey::AppendOnly.as_ref().to_string(), + Some(append.to_string()), + )])), + ), + ] + }; + + let checker = ProtocolChecker::new(HashSet::new(), WRITER_V2.clone()); + + let actions = create_actions(1, "true", vec![]); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let eager = snapshot.snapshot(); + assert!(checker + .can_commit(eager, &append_actions, &append_op) + .is_ok()); + assert!(checker + .can_commit(eager, &change_actions, &change_op) + .is_ok()); + assert!(checker + .can_commit(eager, &neutral_actions, &neutral_op) + .is_ok()); + + let actions = create_actions(2, "true", vec![]); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let eager = snapshot.snapshot(); + assert!(checker + .can_commit(eager, &append_actions, &append_op) + .is_ok()); + assert!(checker + .can_commit(eager, &change_actions, &change_op) + .is_err()); + assert!(checker + .can_commit(eager, &neutral_actions, &neutral_op) + .is_ok()); + + let actions = create_actions(2, "false", vec![]); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let eager = snapshot.snapshot(); + assert!(checker + .can_commit(eager, &append_actions, &append_op) + .is_ok()); + assert!(checker + .can_commit(eager, &change_actions, &change_op) + .is_ok()); + assert!(checker + .can_commit(eager, &neutral_actions, &neutral_op) + .is_ok()); + + let actions = create_actions(7, "true", vec![WriterFeatures::AppendOnly]); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let eager = snapshot.snapshot(); + assert!(checker + .can_commit(eager, &append_actions, &append_op) + .is_ok()); + assert!(checker + .can_commit(eager, &change_actions, &change_op) + .is_err()); + assert!(checker + .can_commit(eager, &neutral_actions, &neutral_op) + .is_ok()); + + let actions = create_actions(7, "false", vec![WriterFeatures::AppendOnly]); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let eager = snapshot.snapshot(); + assert!(checker + .can_commit(eager, &append_actions, &append_op) + .is_ok()); + assert!(checker + .can_commit(eager, &change_actions, &change_op) + .is_ok()); + assert!(checker + .can_commit(eager, &neutral_actions, &neutral_op) + .is_ok()); + + let actions = create_actions(7, "true", vec![]); + let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let eager = snapshot.snapshot(); + assert!(checker + .can_commit(eager, &append_actions, &append_op) + .is_ok()); + assert!(checker + .can_commit(eager, &change_actions, &change_op) + .is_ok()); + assert!(checker + .can_commit(eager, &neutral_actions, &neutral_op) + .is_ok()); + } + + #[test] + fn test_versions() { + let checker_1 = ProtocolChecker::new(HashSet::new(), HashSet::new()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 1, + min_writer_version: 1, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_1 = DeltaTableState::from_actions(actions).unwrap(); + let eager_1 = snapshot_1.snapshot(); + assert!(checker_1.can_read_from(eager_1).is_ok()); + assert!(checker_1.can_write_to(eager_1).is_ok()); + + let checker_2 = ProtocolChecker::new(READER_V2.clone(), HashSet::new()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 1, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_2 = DeltaTableState::from_actions(actions).unwrap(); + let eager_2 = snapshot_2.snapshot(); + assert!(checker_1.can_read_from(eager_2).is_err()); + assert!(checker_1.can_write_to(eager_2).is_err()); + assert!(checker_2.can_read_from(eager_1).is_ok()); + assert!(checker_2.can_read_from(eager_2).is_ok()); + assert!(checker_2.can_write_to(eager_2).is_ok()); + + let checker_3 = ProtocolChecker::new(READER_V2.clone(), WRITER_V2.clone()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 2, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_3 = DeltaTableState::from_actions(actions).unwrap(); + let eager_3 = snapshot_3.snapshot(); + assert!(checker_1.can_read_from(eager_3).is_err()); + assert!(checker_1.can_write_to(eager_3).is_err()); + assert!(checker_2.can_read_from(eager_3).is_ok()); + assert!(checker_2.can_write_to(eager_3).is_err()); + assert!(checker_3.can_read_from(eager_1).is_ok()); + assert!(checker_3.can_read_from(eager_2).is_ok()); + assert!(checker_3.can_read_from(eager_3).is_ok()); + assert!(checker_3.can_write_to(eager_3).is_ok()); + + let checker_4 = ProtocolChecker::new(READER_V2.clone(), WRITER_V3.clone()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 3, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_4 = DeltaTableState::from_actions(actions).unwrap(); + let eager_4 = snapshot_4.snapshot(); + assert!(checker_1.can_read_from(eager_4).is_err()); + assert!(checker_1.can_write_to(eager_4).is_err()); + assert!(checker_2.can_read_from(eager_4).is_ok()); + assert!(checker_2.can_write_to(eager_4).is_err()); + assert!(checker_3.can_read_from(eager_4).is_ok()); + assert!(checker_3.can_write_to(eager_4).is_err()); + assert!(checker_4.can_read_from(eager_1).is_ok()); + assert!(checker_4.can_read_from(eager_2).is_ok()); + assert!(checker_4.can_read_from(eager_3).is_ok()); + assert!(checker_4.can_read_from(eager_4).is_ok()); + assert!(checker_4.can_write_to(eager_4).is_ok()); + + let checker_5 = ProtocolChecker::new(READER_V2.clone(), WRITER_V4.clone()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 4, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_5 = DeltaTableState::from_actions(actions).unwrap(); + let eager_5 = snapshot_5.snapshot(); + assert!(checker_1.can_read_from(eager_5).is_err()); + assert!(checker_1.can_write_to(eager_5).is_err()); + assert!(checker_2.can_read_from(eager_5).is_ok()); + assert!(checker_2.can_write_to(eager_5).is_err()); + assert!(checker_3.can_read_from(eager_5).is_ok()); + assert!(checker_3.can_write_to(eager_5).is_err()); + assert!(checker_4.can_read_from(eager_5).is_ok()); + assert!(checker_4.can_write_to(eager_5).is_err()); + assert!(checker_5.can_read_from(eager_1).is_ok()); + assert!(checker_5.can_read_from(eager_2).is_ok()); + assert!(checker_5.can_read_from(eager_3).is_ok()); + assert!(checker_5.can_read_from(eager_4).is_ok()); + assert!(checker_5.can_read_from(eager_5).is_ok()); + assert!(checker_5.can_write_to(eager_5).is_ok()); + + let checker_6 = ProtocolChecker::new(READER_V2.clone(), WRITER_V5.clone()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 5, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_6 = DeltaTableState::from_actions(actions).unwrap(); + let eager_6 = snapshot_6.snapshot(); + assert!(checker_1.can_read_from(eager_6).is_err()); + assert!(checker_1.can_write_to(eager_6).is_err()); + assert!(checker_2.can_read_from(eager_6).is_ok()); + assert!(checker_2.can_write_to(eager_6).is_err()); + assert!(checker_3.can_read_from(eager_6).is_ok()); + assert!(checker_3.can_write_to(eager_6).is_err()); + assert!(checker_4.can_read_from(eager_6).is_ok()); + assert!(checker_4.can_write_to(eager_6).is_err()); + assert!(checker_5.can_read_from(eager_6).is_ok()); + assert!(checker_5.can_write_to(eager_6).is_err()); + assert!(checker_6.can_read_from(eager_1).is_ok()); + assert!(checker_6.can_read_from(eager_2).is_ok()); + assert!(checker_6.can_read_from(eager_3).is_ok()); + assert!(checker_6.can_read_from(eager_4).is_ok()); + assert!(checker_6.can_read_from(eager_5).is_ok()); + assert!(checker_6.can_read_from(eager_6).is_ok()); + assert!(checker_6.can_write_to(eager_6).is_ok()); + + let checker_7 = ProtocolChecker::new(READER_V2.clone(), WRITER_V6.clone()); + let actions = vec![ + Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 6, + ..Default::default() + }), + create_metadata_action(None, Some(HashMap::new())), + ]; + let snapshot_7 = DeltaTableState::from_actions(actions).unwrap(); + let eager_7 = snapshot_7.snapshot(); + assert!(checker_1.can_read_from(eager_7).is_err()); + assert!(checker_1.can_write_to(eager_7).is_err()); + assert!(checker_2.can_read_from(eager_7).is_ok()); + assert!(checker_2.can_write_to(eager_7).is_err()); + assert!(checker_3.can_read_from(eager_7).is_ok()); + assert!(checker_3.can_write_to(eager_7).is_err()); + assert!(checker_4.can_read_from(eager_7).is_ok()); + assert!(checker_4.can_write_to(eager_7).is_err()); + assert!(checker_5.can_read_from(eager_7).is_ok()); + assert!(checker_5.can_write_to(eager_7).is_err()); + assert!(checker_6.can_read_from(eager_7).is_ok()); + assert!(checker_6.can_write_to(eager_7).is_err()); + assert!(checker_7.can_read_from(eager_1).is_ok()); + assert!(checker_7.can_read_from(eager_2).is_ok()); + assert!(checker_7.can_read_from(eager_3).is_ok()); + assert!(checker_7.can_read_from(eager_4).is_ok()); + assert!(checker_7.can_read_from(eager_5).is_ok()); + assert!(checker_7.can_read_from(eager_6).is_ok()); + assert!(checker_7.can_read_from(eager_7).is_ok()); + assert!(checker_7.can_write_to(eager_7).is_ok()); + } +} diff --git a/crates/deltalake-core/src/operations/transaction/state.rs b/crates/core/src/operations/transaction/state.rs similarity index 70% rename from crates/deltalake-core/src/operations/transaction/state.rs rename to crates/core/src/operations/transaction/state.rs index e64f60d917..8f21018364 100644 --- a/crates/deltalake-core/src/operations/transaction/state.rs +++ b/crates/core/src/operations/transaction/state.rs @@ -1,115 +1,28 @@ +use std::collections::HashSet; use std::sync::Arc; -use arrow::array::ArrayRef; +use arrow::array::{ArrayRef, BooleanArray}; use arrow::datatypes::{ DataType, Field as ArrowField, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, }; -use datafusion::datasource::physical_plan::wrap_partition_type_in_dict; -use datafusion::execution::context::SessionState; -use datafusion::optimizer::utils::conjunction; use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion_common::scalar::ScalarValue; -use datafusion_common::{Column, DFSchema}; +use datafusion_common::Column; use datafusion_expr::Expr; -use itertools::Either; +use itertools::Itertools; use object_store::ObjectStore; +use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; -use crate::delta_datafusion::expr::parse_predicate_expression; use crate::delta_datafusion::{ get_null_of_arrow_type, logical_expr_to_physical_expr, to_correct_scalar_value, + DataFusionMixins, }; -use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::Add; +use crate::errors::DeltaResult; +use crate::kernel::{Add, EagerSnapshot}; use crate::table::state::DeltaTableState; impl DeltaTableState { - /// Get the table schema as an [`ArrowSchemaRef`] - pub fn arrow_schema(&self) -> DeltaResult { - self._arrow_schema(true) - } - - fn _arrow_schema(&self, wrap_partitions: bool) -> DeltaResult { - let meta = self.metadata().ok_or(DeltaTableError::NoMetadata)?; - let fields = meta - .schema - .fields() - .iter() - .filter(|f| !meta.partition_columns.contains(&f.name().to_string())) - .map(|f| f.try_into()) - .chain( - meta.schema - .fields() - .iter() - .filter(|f| meta.partition_columns.contains(&f.name().to_string())) - .map(|f| { - let field = ArrowField::try_from(f)?; - let corrected = if wrap_partitions { - match field.data_type() { - // Only dictionary-encode types that may be large - // // https://github.com/apache/arrow-datafusion/pull/5545 - DataType::Utf8 - | DataType::LargeUtf8 - | DataType::Binary - | DataType::LargeBinary => { - wrap_partition_type_in_dict(field.data_type().clone()) - } - _ => field.data_type().clone(), - } - } else { - field.data_type().clone() - }; - Ok(field.with_data_type(corrected)) - }), - ) - .collect::, _>>()?; - - Ok(Arc::new(ArrowSchema::new(fields))) - } - - pub(crate) fn input_schema(&self) -> DeltaResult { - self._arrow_schema(false) - } - - /// Iterate over all files in the log matching a predicate - pub fn files_matching_predicate( - &self, - filters: &[Expr], - ) -> DeltaResult> { - if let Some(Some(predicate)) = - (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) - { - let expr = logical_expr_to_physical_expr(&predicate, self.arrow_schema()?.as_ref()); - let pruning_predicate = PruningPredicate::try_new(expr, self.arrow_schema()?)?; - Ok(Either::Left( - self.files() - .iter() - .zip(pruning_predicate.prune(self)?) - .filter_map( - |(action, keep_file)| { - if keep_file { - Some(action) - } else { - None - } - }, - ), - )) - } else { - Ok(Either::Right(self.files().iter())) - } - } - - /// Parse an expression string into a datafusion [`Expr`] - pub fn parse_predicate_expression( - &self, - expr: impl AsRef, - df_state: &SessionState, - ) -> DeltaResult { - let schema = DFSchema::try_from(self.arrow_schema()?.as_ref().to_owned())?; - parse_predicate_expression(&schema, expr, df_state) - } - /// Get the physical table schema. /// /// This will construct a schema derived from the parquet schema of the latest data file, @@ -118,14 +31,21 @@ impl DeltaTableState { &self, object_store: Arc, ) -> DeltaResult { - if let Some(add) = self.files().iter().max_by_key(|obj| obj.modification_time) { + if let Some(add) = self + .file_actions()? + .iter() + .max_by_key(|obj| obj.modification_time) + { let file_meta = add.try_into()?; let file_reader = ParquetObjectReader::new(object_store, file_meta); - let file_schema = ParquetRecordBatchStreamBuilder::new(file_reader) - .await? - .build()? - .schema() - .clone(); + let file_schema = ParquetRecordBatchStreamBuilder::new_with_options( + file_reader, + ArrowReaderOptions::new().with_skip_arrow_metadata(true), + ) + .await? + .build()? + .schema() + .clone(); let table_schema = Arc::new(ArrowSchema::new( self.arrow_schema()? @@ -293,31 +213,37 @@ impl<'a> PruningStatistics for AddContainer<'a> { }); ScalarValue::iter_to_array(values).ok() } + + // This function is required since DataFusion 35.0, but is implemented as a no-op + // https://github.com/apache/arrow-datafusion/blob/ec6abece2dcfa68007b87c69eefa6b0d7333f628/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs#L550 + fn contained(&self, _column: &Column, _value: &HashSet) -> Option { + None + } } -impl PruningStatistics for DeltaTableState { +impl PruningStatistics for EagerSnapshot { /// return the minimum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows fn min_values(&self, column: &Column) -> Option { - let partition_columns = &self.metadata()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); + let files = self.file_actions().ok()?.collect_vec(); + let partition_columns = &self.metadata().partition_columns; + let container = AddContainer::new(&files, partition_columns, self.arrow_schema().ok()?); container.min_values(column) } /// return the maximum values for the named column, if known. /// Note: the returned array must contain `num_containers()` rows. fn max_values(&self, column: &Column) -> Option { - let partition_columns = &self.metadata()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); + let files = self.file_actions().ok()?.collect_vec(); + let partition_columns = &self.metadata().partition_columns; + let container = AddContainer::new(&files, partition_columns, self.arrow_schema().ok()?); container.max_values(column) } /// return the number of containers (e.g. row groups) being /// pruned with these statistics fn num_containers(&self) -> usize { - self.files().len() + self.files_count() } /// return the number of null values for the named column as an @@ -325,23 +251,52 @@ impl PruningStatistics for DeltaTableState { /// /// Note: the returned array must contain `num_containers()` rows. fn null_counts(&self, column: &Column) -> Option { - let partition_columns = &self.metadata()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); + let files = self.file_actions().ok()?.collect_vec(); + let partition_columns = &self.metadata().partition_columns; + let container = AddContainer::new(&files, partition_columns, self.arrow_schema().ok()?); container.null_counts(column) } + + // This function is required since DataFusion 35.0, but is implemented as a no-op + // https://github.com/apache/arrow-datafusion/blob/ec6abece2dcfa68007b87c69eefa6b0d7333f628/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs#L550 + fn contained(&self, _column: &Column, _value: &HashSet) -> Option { + None + } +} + +impl PruningStatistics for DeltaTableState { + fn min_values(&self, column: &Column) -> Option { + self.snapshot.min_values(column) + } + + fn max_values(&self, column: &Column) -> Option { + self.snapshot.max_values(column) + } + + fn num_containers(&self) -> usize { + self.snapshot.num_containers() + } + + fn null_counts(&self, column: &Column) -> Option { + self.snapshot.null_counts(column) + } + + fn contained(&self, column: &Column, values: &HashSet) -> Option { + self.snapshot.contained(column, values) + } } #[cfg(test)] mod tests { use super::*; + use crate::delta_datafusion::DataFusionFileMixins; use crate::operations::transaction::test_utils::{create_add_action, init_table_actions}; use datafusion::prelude::SessionContext; use datafusion_expr::{col, lit}; #[test] fn test_parse_predicate_expression() { - let snapshot = DeltaTableState::from_actions(init_table_actions(None), 0).unwrap(); + let snapshot = DeltaTableState::from_actions(init_table_actions(None)).unwrap(); let session = SessionContext::new(); let state = session.state(); @@ -373,8 +328,9 @@ mod tests { actions.push(create_add_action("included-1", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":100},\"nullCount\":{\"value\":0}}".into()))); actions.push(create_add_action("included-2", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":-10},\"maxValues\":{\"value\":3},\"nullCount\":{\"value\":0}}".into()))); - let state = DeltaTableState::from_actions(actions, 0).unwrap(); + let state = DeltaTableState::from_actions(actions).unwrap(); let files = state + .snapshot .files_matching_predicate(&[]) .unwrap() .collect::>(); @@ -385,6 +341,7 @@ mod tests { .or(col("value").lt_eq(lit::(0))); let files = state + .snapshot .files_matching_predicate(&[predictate]) .unwrap() .collect::>(); diff --git a/crates/deltalake-core/src/operations/transaction/test_utils.rs b/crates/core/src/operations/transaction/test_utils.rs similarity index 85% rename from crates/deltalake-core/src/operations/transaction/test_utils.rs rename to crates/core/src/operations/transaction/test_utils.rs index ccb0e090f0..96b0f46323 100644 --- a/crates/deltalake-core/src/operations/transaction/test_utils.rs +++ b/crates/core/src/operations/transaction/test_utils.rs @@ -1,7 +1,7 @@ #![allow(unused)] use std::collections::HashMap; -use super::prepare_commit; +use super::CommitBuilder; use crate::kernel::{ Action, Add, CommitInfo, DataType, Metadata, PrimitiveType, Protocol, Remove, StructField, StructType, @@ -9,7 +9,6 @@ use crate::kernel::{ use crate::operations::transaction::PROTOCOL; use crate::protocol::{DeltaOperation, SaveMode}; use crate::table::state::DeltaTableState; -use crate::table::DeltaTableMetaData; use crate::{DeltaTable, DeltaTableBuilder}; pub fn create_add_action( @@ -24,7 +23,6 @@ pub fn create_add_action( stats, modification_time: -1, partition_values: Default::default(), - partition_values_parsed: None, stats_parsed: None, base_row_id: None, default_row_commit_version: None, @@ -80,15 +78,14 @@ pub fn create_metadata_action( true, ), ]); - let metadata = DeltaTableMetaData::new( - None, - None, - None, - table_schema, - parttiton_columns.unwrap_or_default(), - configuration.unwrap_or_default(), - ); - Action::Metadata(Metadata::try_from(metadata).unwrap()) + Action::Metadata( + Metadata::try_new( + table_schema, + parttiton_columns.unwrap_or_default(), + configuration.unwrap_or_default(), + ) + .unwrap(), + ) } pub fn init_table_actions(configuration: Option>>) -> Vec { @@ -143,7 +140,7 @@ pub async fn create_initialized_table( true, ), ]); - let state = DeltaTableState::from_actions(init_table_actions(None), 0).unwrap(); + let state = DeltaTableState::from_actions(init_table_actions(None)).unwrap(); let operation = DeltaOperation::Create { mode: SaveMode::ErrorIfExists, location: "location".into(), @@ -153,27 +150,22 @@ pub async fn create_initialized_table( writer_features: None, reader_features: None, }, - metadata: DeltaTableMetaData::new( - None, - None, - None, - table_schema, - partition_cols.to_vec(), - configuration.unwrap_or_default(), - ), + metadata: Metadata { + id: uuid::Uuid::new_v4().to_string(), + name: None, + description: None, + format: Default::default(), + schema_string: serde_json::to_string(&table_schema).unwrap(), + partition_columns: partition_cols.to_vec(), + configuration: configuration.unwrap_or_default(), + created_time: Some(chrono::Utc::now().timestamp_millis()), + }, }; let actions = init_table_actions(None); - let prepared_commit = prepare_commit( - log_store.object_store().as_ref(), - &operation, - &actions, - None, - ) - .await - .unwrap(); - - log_store - .write_commit_entry(0, &prepared_commit) + CommitBuilder::default() + .with_actions(actions) + .build(None, log_store.clone(), operation) + .unwrap() .await .unwrap(); DeltaTable::new_with_state(log_store, state) diff --git a/crates/deltalake-core/src/operations/update.rs b/crates/core/src/operations/update.rs similarity index 82% rename from crates/deltalake-core/src/operations/update.rs rename to crates/core/src/operations/update.rs index b7ff813712..6f8b595029 100644 --- a/crates/deltalake-core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -41,18 +41,23 @@ use datafusion_physical_expr::{ use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; use serde::Serialize; -use serde_json::Value; -use super::datafusion_utils::Expression; -use super::transaction::{commit, PROTOCOL}; +use super::transaction::PROTOCOL; use super::write::write_execution_plan; -use crate::delta_datafusion::{expr::fmt_expr_to_sql, physical::MetricObserverExec}; +use super::{ + datafusion_utils::Expression, + transaction::{CommitBuilder, CommitProperties}, +}; +use crate::delta_datafusion::{ + expr::fmt_expr_to_sql, physical::MetricObserverExec, DataFusionMixins, DeltaColumn, + DeltaSessionContext, +}; use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::kernel::{Action, Remove}; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; -use crate::{DeltaResult, DeltaTable, DeltaTableError}; +use crate::{DeltaResult, DeltaTable}; /// Updates records in the Delta Table. /// See this module's documentation for more information @@ -69,8 +74,8 @@ pub struct UpdateBuilder { state: Option, /// Properties passed to underlying parquet writer for when files are rewritten writer_properties: Option, - /// Additional metadata to be added to commit - app_metadata: Option>, + /// Additional information to add to the commit + commit_properties: CommitProperties, /// safe_cast determines how data types that do not match the underlying table are handled /// By default an error is returned safe_cast: bool, @@ -103,7 +108,7 @@ impl UpdateBuilder { log_store, state: None, writer_properties: None, - app_metadata: None, + commit_properties: CommitProperties::default(), safe_cast: false, } } @@ -115,12 +120,12 @@ impl UpdateBuilder { } /// Perform an additional update expression during the operaton - pub fn with_update, E: Into>( + pub fn with_update, E: Into>( mut self, column: S, expression: E, ) -> Self { - self.updates.insert(column.into(), expression.into()); + self.updates.insert(column.into().into(), expression.into()); self } @@ -131,11 +136,8 @@ impl UpdateBuilder { } /// Additional metadata to be added to commit info - pub fn with_metadata( - mut self, - metadata: impl IntoIterator, - ) -> Self { - self.app_metadata = Some(HashMap::from_iter(metadata)); + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; self } @@ -168,9 +170,9 @@ async fn execute( snapshot: &DeltaTableState, state: SessionState, writer_properties: Option, - app_metadata: Option>, + mut commit_properties: CommitProperties, safe_cast: bool, -) -> DeltaResult<((Vec, i64), UpdateMetrics)> { +) -> DeltaResult<((Vec, i64, Option), UpdateMetrics)> { // Validate the predicate and update expressions. // // If the predicate is not set, then all files need to be updated. @@ -183,10 +185,10 @@ async fn execute( let exec_start = Instant::now(); let mut metrics = UpdateMetrics::default(); - let mut version = snapshot.version(); + let version = snapshot.version(); if updates.is_empty() { - return Ok(((Vec::new(), version), metrics)); + return Ok(((Vec::new(), version, None), metrics)); } let predicate = match predicate { @@ -207,7 +209,7 @@ async fn execute( }) .collect::, _>>()?; - let current_metadata = snapshot.metadata().ok_or(DeltaTableError::NoMetadata)?; + let current_metadata = snapshot.metadata(); let table_partition_cols = current_metadata.partition_columns.clone(); let scan_start = Instant::now(); @@ -215,7 +217,7 @@ async fn execute( metrics.scan_time_ms = Instant::now().duration_since(scan_start).as_millis() as u64; if candidates.candidates.is_empty() { - return Ok(((Vec::new(), version), metrics)); + return Ok(((Vec::new(), version, None), metrics)); } let predicate = predicate.unwrap_or(Expr::Literal(ScalarValue::Boolean(Some(true)))); @@ -261,12 +263,7 @@ async fn execute( let predicate_null = when(predicate.clone(), lit(true)).otherwise(lit(ScalarValue::Boolean(None)))?; - let predicate_expr = create_physical_expr( - &predicate_null, - &input_dfschema, - &input_schema, - execution_props, - )?; + let predicate_expr = create_physical_expr(&predicate_null, &input_dfschema, execution_props)?; expressions.push((predicate_expr, "__delta_rs_update_predicate".to_string())); let projection_predicate: Arc = @@ -313,8 +310,7 @@ async fn execute( let expr = case(col("__delta_rs_update_predicate")) .when(lit(true), expr.to_owned()) .otherwise(col(column.to_owned()))?; - let predicate_expr = - create_physical_expr(&expr, &input_dfschema, &input_schema, execution_props)?; + let predicate_expr = create_physical_expr(&expr, &input_dfschema, execution_props)?; map.insert(column.name.clone(), expressions.len()); let c = "__delta_rs_".to_string() + &column.name; expressions.push((predicate_expr, c.clone())); @@ -352,7 +348,7 @@ async fn execute( )?); let add_actions = write_execution_plan( - snapshot, + Some(snapshot), state.clone(), projection.clone(), table_partition_cols.clone(), @@ -361,7 +357,7 @@ async fn execute( None, writer_properties, safe_cast, - false, + None, ) .await?; @@ -381,7 +377,7 @@ async fn execute( .duration_since(UNIX_EPOCH) .unwrap() .as_millis() as i64; - let mut actions: Vec = add_actions.into_iter().map(Action::Add).collect(); + let mut actions: Vec = add_actions.clone(); metrics.num_added_files = actions.len(); metrics.num_removed_files = candidates.candidates.len(); @@ -406,16 +402,29 @@ async fn execute( let operation = DeltaOperation::Update { predicate: Some(fmt_expr_to_sql(&predicate)?), }; - version = commit( - log_store.as_ref(), - &actions, - operation, - snapshot, - app_metadata, - ) - .await?; - Ok(((actions, version), metrics)) + commit_properties + .app_metadata + .insert("readVersion".to_owned(), snapshot.version().into()); + + commit_properties.app_metadata.insert( + "operationMetrics".to_owned(), + serde_json::to_value(&metrics)?, + ); + + let commit = CommitBuilder::from(commit_properties) + .with_actions(actions) + .build(Some(snapshot), log_store, operation)? + .await?; + + Ok(( + ( + commit.data.actions, + commit.version, + Some(commit.data.operation), + ), + metrics, + )) } impl std::future::IntoFuture for UpdateBuilder { @@ -426,12 +435,11 @@ impl std::future::IntoFuture for UpdateBuilder { let mut this = self; Box::pin(async move { - PROTOCOL.check_append_only(&this.snapshot)?; - - PROTOCOL.can_write_to(&this.snapshot)?; + PROTOCOL.check_append_only(&this.snapshot.snapshot)?; + PROTOCOL.can_write_to(&this.snapshot.snapshot)?; let state = this.state.unwrap_or_else(|| { - let session = SessionContext::new(); + let session: SessionContext = DeltaSessionContext::default().into(); // If a user provides their own their DF state then they must register the store themselves register_store(this.log_store.clone(), session.runtime_env()); @@ -439,22 +447,23 @@ impl std::future::IntoFuture for UpdateBuilder { session.state() }); - let ((actions, version), metrics) = execute( + let ((actions, version, operation), metrics) = execute( this.predicate, this.updates, this.log_store.clone(), &this.snapshot, state, this.writer_properties, - this.app_metadata, + this.commit_properties, this.safe_cast, ) .await?; - this.snapshot - .merge(DeltaTableState::from_actions(actions, version)?, true, true); - let table = DeltaTable::new_with_state(this.log_store, this.snapshot); + if let Some(op) = &operation { + this.snapshot.merge(actions, op, version)?; + } + let table = DeltaTable::new_with_state(this.log_store, this.snapshot); Ok((table, metrics)) }) } @@ -462,6 +471,10 @@ impl std::future::IntoFuture for UpdateBuilder { #[cfg(test)] mod tests { + use crate::kernel::DataType as DeltaDataType; + use crate::kernel::PrimitiveType; + use crate::kernel::StructField; + use crate::kernel::StructType; use crate::operations::DeltaOps; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::datafusion::write_batch; @@ -470,9 +483,11 @@ mod tests { }; use crate::DeltaConfigKey; use crate::DeltaTable; + use arrow::datatypes::Schema as ArrowSchema; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; use arrow_array::Int32Array; + use arrow_schema::DataType; use datafusion::assert_batches_sorted_eq; use datafusion::prelude::*; use serde_json::json; @@ -548,7 +563,7 @@ mod tests { let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) .update() @@ -557,7 +572,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 4); @@ -602,9 +617,9 @@ mod tests { let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); - let (mut table, metrics) = DeltaOps(table) + let (table, metrics) = DeltaOps(table) .update() .with_predicate(col("modified").eq(lit("2021-02-03"))) .with_update("modified", lit("2023-05-14")) @@ -612,14 +627,14 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); assert_eq!(metrics.num_copied_rows, 2); let commit_info = table.history(None).await.unwrap(); - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["predicate"], json!("modified = '2021-02-03'")); @@ -659,7 +674,7 @@ mod tests { let table = write_batch(table, batch.clone()).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .update() @@ -670,7 +685,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); @@ -694,7 +709,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) .update() @@ -709,7 +724,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 3); + assert_eq!(table.get_files_count(), 3); assert_eq!(metrics.num_added_files, 2); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 1); @@ -730,11 +745,82 @@ mod tests { assert_batches_sorted_eq!(&expected, &actual); } + #[tokio::test] + async fn test_update_case_sensitive() { + let schema = StructType::new(vec![ + StructField::new( + "Id".to_string(), + DeltaDataType::Primitive(PrimitiveType::String), + true, + ), + StructField::new( + "ValUe".to_string(), + DeltaDataType::Primitive(PrimitiveType::Integer), + true, + ), + StructField::new( + "mOdified".to_string(), + DeltaDataType::Primitive(PrimitiveType::String), + true, + ), + ]); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("Id", DataType::Utf8, true), + Field::new("ValUe", DataType::Int32, true), + Field::new("mOdified", DataType::Utf8, true), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&arrow_schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "A", "A"])), + Arc::new(arrow::array::Int32Array::from(vec![1, 10, 10, 100])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2021-02-02", + "2021-02-03", + "2021-02-03", + ])), + ], + ) + .unwrap(); + + let table = DeltaOps::new_in_memory() + .create() + .with_columns(schema.fields().clone()) + .await + .unwrap(); + let table = write_batch(table, batch).await; + + let (table, _metrics) = DeltaOps(table) + .update() + .with_predicate("mOdified = '2021-02-03'") + .with_update("mOdified", "'2023-05-14'") + .with_update("Id", "'C'") + .await + .unwrap(); + + let expected = vec![ + "+----+-------+------------+", + "| Id | ValUe | mOdified |", + "+----+-------+------------+", + "| A | 1 | 2021-02-02 |", + "| B | 10 | 2021-02-02 |", + "| C | 10 | 2023-05-14 |", + "| C | 100 | 2023-05-14 |", + "+----+-------+------------+", + ]; + + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } + #[tokio::test] async fn test_update_null() { let table = prepare_values_table().await; assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) .update() @@ -742,7 +828,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 5); @@ -772,12 +858,20 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); assert_eq!(metrics.num_copied_rows, 3); + let commit_info = table.history(None).await.unwrap(); + let last_commit = &commit_info[0]; + let extra_info = last_commit.info.clone(); + assert_eq!( + extra_info["operationMetrics"], + serde_json::to_value(&metrics).unwrap() + ); + let expected = [ "+-------+", "| value |", @@ -800,7 +894,7 @@ mod tests { .await .unwrap(); assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); assert_eq!(metrics.num_updated_rows, 2); diff --git a/crates/deltalake-core/src/operations/vacuum.rs b/crates/core/src/operations/vacuum.rs similarity index 79% rename from crates/deltalake-core/src/operations/vacuum.rs rename to crates/core/src/operations/vacuum.rs index 7b321400e6..fd986b55d6 100644 --- a/crates/deltalake-core/src/operations/vacuum.rs +++ b/crates/core/src/operations/vacuum.rs @@ -21,7 +21,7 @@ //! let (table, metrics) = VacuumBuilder::new(table.object_store(). table.state).await?; //! ```` -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; @@ -31,13 +31,10 @@ use futures::{StreamExt, TryStreamExt}; use object_store::Error; use object_store::{path::Path, ObjectStore}; use serde::Serialize; -use serde_json::Value; -use super::transaction::commit; -use crate::crate_version; +use super::transaction::{CommitBuilder, CommitProperties}; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::Action; -use crate::logstore::{LogStore, LogStoreRef}; +use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::DeltaTable; @@ -59,6 +56,9 @@ enum VacuumError { /// Error returned #[error(transparent)] DeltaTable(#[from] DeltaTableError), + + #[error(transparent)] + Protocol(#[from] crate::protocol::ProtocolError), } impl From for DeltaTableError { @@ -91,6 +91,8 @@ pub struct VacuumBuilder { dry_run: bool, /// Override the source of time clock: Option>, + /// Additional information to add to the commit + commit_properties: CommitProperties, } /// Details for the Vacuum operation including which files were @@ -133,6 +135,7 @@ impl VacuumBuilder { enforce_retention_duration: true, dry_run: false, clock: None, + commit_properties: CommitProperties::default(), } } @@ -161,6 +164,12 @@ impl VacuumBuilder { self } + /// Additional metadata to be added to commit info + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + self + } + /// Determine which files can be deleted. Does not actually peform the deletion async fn create_vacuum_plan(&self) -> Result { let min_retention = Duration::milliseconds( @@ -184,21 +193,20 @@ impl VacuumBuilder { None => Utc::now().timestamp_millis(), }; - let expired_tombstones = get_stale_files(&self.snapshot, retention_period, now_millis); + let expired_tombstones = get_stale_files( + &self.snapshot, + retention_period, + now_millis, + self.log_store.object_store().clone(), + ) + .await?; let valid_files = self.snapshot.file_paths_iter().collect::>(); let mut files_to_delete = vec![]; let mut file_sizes = vec![]; let object_store = self.log_store.object_store(); - let mut all_files = object_store - .list(None) - .await - .map_err(DeltaTableError::from)?; - let partition_columns = &self - .snapshot - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns; + let mut all_files = object_store.list(None); + let partition_columns = &self.snapshot.metadata().partition_columns; while let Some(obj_meta) = all_files.next().await { // TODO should we allow NotFound here in case we have a temporary commit file in the list @@ -244,7 +252,11 @@ impl std::future::IntoFuture for VacuumBuilder { } let metrics = plan - .execute(this.log_store.as_ref(), &this.snapshot) + .execute( + this.log_store.clone(), + &this.snapshot, + this.commit_properties, + ) .await?; Ok(( DeltaTable::new_with_state(this.log_store, this.snapshot), @@ -272,8 +284,9 @@ impl VacuumPlan { /// Execute the vacuum plan and delete files from underlying storage pub async fn execute( self, - store: &dyn LogStore, + store: LogStoreRef, snapshot: &DeltaTableState, + mut commit_properties: CommitProperties, ) -> Result { if self.files_to_delete.is_empty() { return Ok(VacuumMetrics { @@ -292,28 +305,22 @@ impl VacuumPlan { status: String::from("COMPLETED"), // Maybe this should be FAILED when vacuum has error during the files, not sure how to check for this }; - let start_metrics = serde_json::to_value(VacuumStartOperationMetrics { + let start_metrics = VacuumStartOperationMetrics { num_files_to_delete: self.files_to_delete.len() as i64, size_of_data_to_delete: self.file_sizes.iter().sum(), - }); + }; // Begin VACUUM START COMMIT - let mut commit_info = start_operation.get_commit_info(); - let mut extra_info = HashMap::::new(); - - commit_info.timestamp = Some(Utc::now().timestamp_millis()); - extra_info.insert( - "clientVersion".to_string(), - Value::String(format!("delta-rs.{}", crate_version())), + let mut start_props = CommitProperties::default(); + start_props.app_metadata = commit_properties.app_metadata.clone(); + start_props.app_metadata.insert( + "operationMetrics".to_owned(), + serde_json::to_value(start_metrics)?, ); - if let Ok(map) = start_metrics { - extra_info.insert("operationMetrics".to_owned(), map); - } - commit_info.info = extra_info; - let start_actions = vec![Action::CommitInfo(commit_info)]; - - commit(store, &start_actions, start_operation, snapshot, None).await?; + CommitBuilder::from(start_props) + .build(Some(snapshot), store.clone(), start_operation)? + .await?; // Finish VACUUM START COMMIT let locations = futures::stream::iter(self.files_to_delete) @@ -332,28 +339,19 @@ impl VacuumPlan { .await?; // Create end metadata - let end_metrics = serde_json::to_value(VacuumEndOperationMetrics { + let end_metrics = VacuumEndOperationMetrics { num_deleted_files: files_deleted.len() as i64, num_vacuumed_directories: 0, // Set to zero since we only remove files not dirs - }); + }; // Begin VACUUM END COMMIT - let mut commit_info = end_operation.get_commit_info(); - let mut extra_info = HashMap::::new(); - - commit_info.timestamp = Some(Utc::now().timestamp_millis()); - extra_info.insert( - "clientVersion".to_string(), - Value::String(format!("delta-rs.{}", crate_version())), + commit_properties.app_metadata.insert( + "operationMetrics".to_owned(), + serde_json::to_value(end_metrics)?, ); - if let Ok(map) = end_metrics { - extra_info.insert("operationMetrics".to_owned(), map); - } - commit_info.info = extra_info; - - let end_actions = vec![Action::CommitInfo(commit_info)]; - - commit(store, &end_actions, end_operation, snapshot, None).await?; + CommitBuilder::from(commit_properties) + .build(Some(snapshot), store.clone(), end_operation)? + .await?; // Finish VACUUM END COMMIT Ok(VacuumMetrics { @@ -378,22 +376,25 @@ fn is_hidden_directory(partition_columns: &[String], path: &Path) -> Result HashSet<&str> { + store: Arc, +) -> DeltaResult> { let tombstone_retention_timestamp = now_timestamp_millis - retention_period.num_milliseconds(); - snapshot - .all_tombstones() - .iter() + Ok(snapshot + .all_tombstones(store) + .await? + .collect::>() + .into_iter() .filter(|tombstone| { // if the file has a creation time before the `tombstone_retention_timestamp` // then it's considered as a stale file tombstone.deletion_timestamp.unwrap_or(0) < tombstone_retention_timestamp }) - .map(|tombstone| tombstone.path.as_str()) - .collect::>() + .map(|tombstone| tombstone.path) + .collect::>()) } #[cfg(test)] @@ -404,33 +405,36 @@ mod tests { #[tokio::test] async fn vacuum_delta_8_0_table() { - let table = open_table("./tests/data/delta-0.8.0").await.unwrap(); + let table = open_table("../test/tests/data/delta-0.8.0").await.unwrap(); - let result = VacuumBuilder::new(table.log_store, table.state.clone()) + let result = VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) .with_retention_period(Duration::hours(1)) .with_dry_run(true) .await; assert!(result.is_err()); - let table = open_table("./tests/data/delta-0.8.0").await.unwrap(); - let (table, result) = VacuumBuilder::new(table.log_store, table.state) - .with_retention_period(Duration::hours(0)) - .with_dry_run(true) - .with_enforce_retention_duration(false) - .await - .unwrap(); + let table = open_table("../test/tests/data/delta-0.8.0").await.unwrap(); + + let (table, result) = + VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) + .with_retention_period(Duration::hours(0)) + .with_dry_run(true) + .with_enforce_retention_duration(false) + .await + .unwrap(); // do not enforce retention duration check with 0 hour will purge all files assert_eq!( result.files_deleted, vec!["part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"] ); - let (table, result) = VacuumBuilder::new(table.log_store, table.state) - .with_retention_period(Duration::hours(169)) - .with_dry_run(true) - .await - .unwrap(); + let (table, result) = + VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) + .with_retention_period(Duration::hours(169)) + .with_dry_run(true) + .await + .unwrap(); assert_eq!( result.files_deleted, @@ -443,11 +447,12 @@ mod tests { .as_secs() / 3600; let empty: Vec = Vec::new(); - let (_table, result) = VacuumBuilder::new(table.log_store, table.state) - .with_retention_period(Duration::hours(retention_hours as i64)) - .with_dry_run(true) - .await - .unwrap(); + let (_table, result) = + VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) + .with_retention_period(Duration::hours(retention_hours as i64)) + .with_dry_run(true) + .await + .unwrap(); assert_eq!(result.files_deleted, empty); } diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs new file mode 100644 index 0000000000..d2751a6b1f --- /dev/null +++ b/crates/core/src/operations/write.rs @@ -0,0 +1,1613 @@ +//! +//! New Table Semantics +//! - The schema of the [RecordBatch] is used to initialize the table. +//! - The partition columns will be used to partition the table. +//! +//! Existing Table Semantics +//! - The save mode will control how existing data is handled (i.e. overwrite, append, etc) +//! - (NOT YET IMPLEMENTED) The schema of the RecordBatch will be checked and if there are new columns present +//! they will be added to the tables schema. Conflicting columns (i.e. a INT, and a STRING) +//! will result in an exception. +//! - The partition columns, if present, are validated against the existing metadata. If not +//! present, then the partitioning of the table is respected. +//! +//! In combination with `Overwrite`, a `replaceWhere` option can be used to transactionally +//! replace data that matches a predicate. +//! +//! # Example +//! ```rust ignore +//! let id_field = arrow::datatypes::Field::new("id", arrow::datatypes::DataType::Int32, false); +//! let schema = Arc::new(arrow::datatypes::Schema::new(vec![id_field])); +//! let ids = arrow::array::Int32Array::from(vec![1, 2, 3, 4, 5]); +//! let batch = RecordBatch::try_new(schema, vec![Arc::new(ids)])?; +//! let ops = DeltaOps::try_from_uri("../path/to/empty/dir").await?; +//! let table = ops.write(vec![batch]).await?; +//! ```` + +use std::collections::HashMap; +use std::str::FromStr; +use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; +use std::vec; + +use arrow_array::RecordBatch; +use arrow_cast::can_cast_types; +use arrow_schema::{ArrowError, DataType, Fields, SchemaRef as ArrowSchemaRef}; +use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; +use datafusion::physical_expr::create_physical_expr; +use datafusion::physical_plan::filter::FilterExec; +use datafusion::physical_plan::{memory::MemoryExec, ExecutionPlan}; +use datafusion_common::DFSchema; +use datafusion_expr::Expr; +use futures::future::BoxFuture; +use futures::StreamExt; +use parquet::file::properties::WriterProperties; + +use super::datafusion_utils::Expression; +use super::transaction::{CommitBuilder, CommitProperties, TableReference, PROTOCOL}; +use super::writer::{DeltaWriter, WriterConfig}; +use super::CreateBuilder; +use crate::delta_datafusion::expr::fmt_expr_to_sql; +use crate::delta_datafusion::expr::parse_predicate_expression; +use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; +use crate::delta_datafusion::{DataFusionMixins, DeltaDataChecker}; +use crate::errors::{DeltaResult, DeltaTableError}; +use crate::kernel::{Action, Add, Metadata, PartitionsExt, Remove, StructType}; +use crate::logstore::LogStoreRef; +use crate::operations::cast::{cast_record_batch, merge_schema}; +use crate::protocol::{DeltaOperation, SaveMode}; +use crate::storage::ObjectStoreRef; +use crate::table::state::DeltaTableState; +use crate::table::Constraint as DeltaConstraint; +use crate::writer::record_batch::divide_by_partition_values; +use crate::DeltaTable; + +#[derive(thiserror::Error, Debug)] +enum WriteError { + #[error("No data source supplied to write command.")] + MissingData, + + #[error("Failed to execute write task: {source}")] + WriteTask { source: tokio::task::JoinError }, + + #[error("A table already exists at: {0}")] + AlreadyExists(String), + + #[error( + "Specified table partitioning does not match table partitioning: expected: {expected:?}, got: {got:?}", + )] + PartitionColumnMismatch { + expected: Vec, + got: Vec, + }, +} + +impl From for DeltaTableError { + fn from(err: WriteError) -> Self { + DeltaTableError::GenericError { + source: Box::new(err), + } + } +} + +///Specifies how to handle schema drifts +#[derive(PartialEq, Clone, Copy)] +pub enum SchemaMode { + /// Overwrite the schema with the new schema + Overwrite, + /// Append the new schema to the existing schema + Merge, +} + +impl FromStr for SchemaMode { + type Err = DeltaTableError; + + fn from_str(s: &str) -> DeltaResult { + match s.to_ascii_lowercase().as_str() { + "overwrite" => Ok(SchemaMode::Overwrite), + "merge" => Ok(SchemaMode::Merge), + _ => Err(DeltaTableError::Generic(format!( + "Invalid schema write mode provided: {}, only these are supported: ['overwrite', 'merge']", + s + ))), + } + } +} + +/// Write data into a DeltaTable +pub struct WriteBuilder { + /// A snapshot of the to-be-loaded table's state + snapshot: Option, + /// Delta object store for handling data files + log_store: LogStoreRef, + /// The input plan + input: Option>, + /// Datafusion session state relevant for executing the input plan + state: Option, + /// SaveMode defines how to treat data already written to table location + mode: SaveMode, + /// Column names for table partitioning + partition_columns: Option>, + /// When using `Overwrite` mode, replace data that matches a predicate + predicate: Option, + /// Size above which we will write a buffered parquet file to disk. + target_file_size: Option, + /// Number of records to be written in single batch to underlying writer + write_batch_size: Option, + /// RecordBatches to be written into the table + batches: Option>, + /// whether to overwrite the schema or to merge it. None means to fail on schmema drift + schema_mode: Option, + /// how to handle cast failures, either return NULL (safe=true) or return ERR (safe=false) + safe_cast: bool, + /// Parquet writer properties + writer_properties: Option, + /// Additional information to add to the commit + commit_properties: CommitProperties, + /// Name of the table, only used when table doesn't exist yet + name: Option, + /// Description of the table, only used when table doesn't exist yet + description: Option, + /// Configurations of the delta table, only used when table doesn't exist + configuration: HashMap>, +} + +impl WriteBuilder { + /// Create a new [`WriteBuilder`] + pub fn new(log_store: LogStoreRef, snapshot: Option) -> Self { + Self { + snapshot, + log_store, + input: None, + state: None, + mode: SaveMode::Append, + partition_columns: None, + predicate: None, + target_file_size: None, + write_batch_size: None, + batches: None, + safe_cast: false, + schema_mode: None, + writer_properties: None, + commit_properties: CommitProperties::default(), + name: None, + description: None, + configuration: Default::default(), + } + } + + /// Specify the behavior when a table exists at location + pub fn with_save_mode(mut self, save_mode: SaveMode) -> Self { + self.mode = save_mode; + self + } + + /// Add Schema Write Mode + pub fn with_schema_mode(mut self, schema_mode: SchemaMode) -> Self { + self.schema_mode = Some(schema_mode); + self + } + + /// When using `Overwrite` mode, replace data that matches a predicate + pub fn with_replace_where(mut self, predicate: impl Into) -> Self { + self.predicate = Some(predicate.into()); + self + } + + /// (Optional) Specify table partitioning. If specified, the partitioning is validated, + /// if the table already exists. In case a new table is created, the partitioning is applied. + pub fn with_partition_columns( + mut self, + partition_columns: impl IntoIterator>, + ) -> Self { + self.partition_columns = Some(partition_columns.into_iter().map(|s| s.into()).collect()); + self + } + + /// Execution plan that produces the data to be written to the delta table + pub fn with_input_execution_plan(mut self, plan: Arc) -> Self { + self.input = Some(plan); + self + } + + /// A session state accompanying a given input plan, containing e.g. registered object stores + pub fn with_input_session_state(mut self, state: SessionState) -> Self { + self.state = Some(state); + self + } + + /// Execution plan that produces the data to be written to the delta table + pub fn with_input_batches(mut self, batches: impl IntoIterator) -> Self { + self.batches = Some(batches.into_iter().collect()); + self + } + + /// Specify the target file size for data files written to the delta table. + pub fn with_target_file_size(mut self, target_file_size: usize) -> Self { + self.target_file_size = Some(target_file_size); + self + } + + /// Specify the target batch size for row groups written to parquet files. + pub fn with_write_batch_size(mut self, write_batch_size: usize) -> Self { + self.write_batch_size = Some(write_batch_size); + self + } + + /// Specify the safety of the casting operation + /// how to handle cast failures, either return NULL (safe=true) or return ERR (safe=false) + pub fn with_cast_safety(mut self, safe: bool) -> Self { + self.safe_cast = safe; + self + } + + /// Specify the writer properties to use when writing a parquet file + pub fn with_writer_properties(mut self, writer_properties: WriterProperties) -> Self { + self.writer_properties = Some(writer_properties); + self + } + + /// Additional metadata to be added to commit info + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + self + } + + /// Specify the table name. Optionally qualified with + /// a database name [database_name.] table_name. + pub fn with_table_name(mut self, name: impl Into) -> Self { + self.name = Some(name.into()); + self + } + + /// Comment to describe the table. + pub fn with_description(mut self, description: impl Into) -> Self { + self.description = Some(description.into()); + self + } + + /// Set configuration on created table + pub fn with_configuration( + mut self, + configuration: impl IntoIterator, Option>)>, + ) -> Self { + self.configuration = configuration + .into_iter() + .map(|(k, v)| (k.into(), v.map(|s| s.into()))) + .collect(); + self + } + + async fn check_preconditions(&self) -> DeltaResult> { + match &self.snapshot { + Some(snapshot) => { + PROTOCOL.can_write_to(snapshot)?; + + if let Some(plan) = &self.input { + let schema: StructType = (plan.schema()).try_into()?; + PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; + } else if let Some(batches) = &self.batches { + if batches.is_empty() { + return Err(WriteError::MissingData.into()); + } + let schema: StructType = (batches[0].schema()).try_into()?; + PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; + } + + match self.mode { + SaveMode::ErrorIfExists => { + Err(WriteError::AlreadyExists(self.log_store.root_uri()).into()) + } + _ => Ok(vec![]), + } + } + None => { + let schema: StructType = if let Some(plan) = &self.input { + Ok(plan.schema().try_into()?) + } else if let Some(batches) = &self.batches { + if batches.is_empty() { + return Err(WriteError::MissingData.into()); + } + Ok(batches[0].schema().try_into()?) + } else { + Err(WriteError::MissingData) + }?; + let mut builder = CreateBuilder::new() + .with_log_store(self.log_store.clone()) + .with_columns(schema.fields().clone()) + .with_configuration(self.configuration.clone()); + if let Some(partition_columns) = self.partition_columns.as_ref() { + builder = builder.with_partition_columns(partition_columns.clone()) + } + + if let Some(name) = self.name.as_ref() { + builder = builder.with_table_name(name.clone()); + }; + + if let Some(desc) = self.description.as_ref() { + builder = builder.with_comment(desc.clone()); + }; + + let (_, actions, _) = builder.into_table_and_actions()?; + Ok(actions) + } + } + } +} + +#[allow(clippy::too_many_arguments)] +async fn write_execution_plan_with_predicate( + predicate: Option, + snapshot: Option<&DeltaTableState>, + state: SessionState, + plan: Arc, + partition_columns: Vec, + object_store: ObjectStoreRef, + target_file_size: Option, + write_batch_size: Option, + writer_properties: Option, + safe_cast: bool, + schema_mode: Option, +) -> DeltaResult> { + let schema: ArrowSchemaRef = if schema_mode.is_some() { + plan.schema() + } else { + snapshot + .and_then(|s| s.input_schema().ok()) + .unwrap_or(plan.schema()) + }; + + let checker = if let Some(snapshot) = snapshot { + DeltaDataChecker::new(snapshot) + } else { + DeltaDataChecker::empty() + }; + let checker = match predicate { + Some(pred) => { + // TODO: get the name of the outer-most column? `*` will also work but would it be slower? + let chk = DeltaConstraint::new("*", &fmt_expr_to_sql(&pred)?); + checker.with_extra_constraints(vec![chk]) + } + _ => checker, + }; + + // Write data to disk + let mut tasks = vec![]; + for i in 0..plan.output_partitioning().partition_count() { + let inner_plan = plan.clone(); + let inner_schema = schema.clone(); + let task_ctx = Arc::new(TaskContext::from(&state)); + let config = WriterConfig::new( + inner_schema.clone(), + partition_columns.clone(), + writer_properties.clone(), + target_file_size, + write_batch_size, + ); + let mut writer = DeltaWriter::new(object_store.clone(), config); + let checker_stream = checker.clone(); + let mut stream = inner_plan.execute(i, task_ctx)?; + let handle: tokio::task::JoinHandle>> = + tokio::task::spawn(async move { + while let Some(maybe_batch) = stream.next().await { + let batch = maybe_batch?; + checker_stream.check_batch(&batch).await?; + let arr = super::cast::cast_record_batch( + &batch, + inner_schema.clone(), + safe_cast, + schema_mode == Some(SchemaMode::Merge), + )?; + writer.write(&arr).await?; + } + let add_actions = writer.close().await; + match add_actions { + Ok(actions) => Ok(actions.into_iter().map(Action::Add).collect::>()), + Err(err) => Err(err), + } + }); + + tasks.push(handle); + } + let actions = futures::future::join_all(tasks) + .await + .into_iter() + .collect::, _>>() + .map_err(|err| WriteError::WriteTask { source: err })? + .into_iter() + .collect::, _>>()? + .concat() + .into_iter() + .collect::>(); + // Collect add actions to add to commit + Ok(actions) +} + +#[allow(clippy::too_many_arguments)] +pub(crate) async fn write_execution_plan( + snapshot: Option<&DeltaTableState>, + state: SessionState, + plan: Arc, + partition_columns: Vec, + object_store: ObjectStoreRef, + target_file_size: Option, + write_batch_size: Option, + writer_properties: Option, + safe_cast: bool, + schema_mode: Option, +) -> DeltaResult> { + write_execution_plan_with_predicate( + None, + snapshot, + state, + plan, + partition_columns, + object_store, + target_file_size, + write_batch_size, + writer_properties, + safe_cast, + schema_mode, + ) + .await +} + +async fn execute_non_empty_expr( + snapshot: &DeltaTableState, + log_store: LogStoreRef, + state: SessionState, + partition_columns: Vec, + expression: &Expr, + rewrite: &[Add], + writer_properties: Option, +) -> DeltaResult> { + // For each identified file perform a parquet scan + filter + limit (1) + count. + // If returned count is not zero then append the file to be rewritten and removed from the log. Otherwise do nothing to the file. + + let input_schema = snapshot.input_schema()?; + let input_dfschema: DFSchema = input_schema.clone().as_ref().clone().try_into()?; + + let scan = DeltaScanBuilder::new(snapshot, log_store.clone(), &state) + .with_files(rewrite) + .build() + .await?; + let scan = Arc::new(scan); + + // Apply the negation of the filter and rewrite files + let negated_expression = Expr::Not(Box::new(Expr::IsTrue(Box::new(expression.clone())))); + + let predicate_expr = create_physical_expr( + &negated_expression, + &input_dfschema, + state.execution_props(), + )?; + let filter: Arc = + Arc::new(FilterExec::try_new(predicate_expr, scan.clone())?); + + // We don't want to verify the predicate against existing data + let add_actions = write_execution_plan( + Some(snapshot), + state, + filter, + partition_columns, + log_store.object_store(), + Some(snapshot.table_config().target_file_size() as usize), + None, + writer_properties, + false, + None, + ) + .await?; + + Ok(add_actions) +} + +// This should only be called wth a valid predicate +async fn prepare_predicate_actions( + predicate: Expr, + log_store: LogStoreRef, + snapshot: &DeltaTableState, + state: SessionState, + partition_columns: Vec, + writer_properties: Option, + deletion_timestamp: i64, +) -> DeltaResult> { + let candidates = + find_files(snapshot, log_store.clone(), &state, Some(predicate.clone())).await?; + + let add = if candidates.partition_scan { + Vec::new() + } else { + execute_non_empty_expr( + snapshot, + log_store, + state, + partition_columns, + &predicate, + &candidates.candidates, + writer_properties, + ) + .await? + }; + let remove = candidates.candidates; + + let mut actions: Vec = add.into_iter().collect(); + + for action in remove { + actions.push(Action::Remove(Remove { + path: action.path, + deletion_timestamp: Some(deletion_timestamp), + data_change: true, + extended_file_metadata: Some(true), + partition_values: Some(action.partition_values), + size: Some(action.size), + deletion_vector: action.deletion_vector, + tags: None, + base_row_id: action.base_row_id, + default_row_commit_version: action.default_row_commit_version, + })) + } + Ok(actions) +} + +impl std::future::IntoFuture for WriteBuilder { + type Output = DeltaResult; + type IntoFuture = BoxFuture<'static, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let this = self; + + Box::pin(async move { + if this.mode == SaveMode::Overwrite { + if let Some(snapshot) = &this.snapshot { + PROTOCOL.check_append_only(&snapshot.snapshot)?; + } + } + if this.schema_mode == Some(SchemaMode::Overwrite) && this.mode != SaveMode::Overwrite { + return Err(DeltaTableError::Generic( + "Schema overwrite not supported for Append".to_string(), + )); + } + + // Create table actions to initialize table in case it does not yet exist and should be created + let mut actions = this.check_preconditions().await?; + + let active_partitions = this + .snapshot + .as_ref() + .map(|s| s.metadata().partition_columns.clone()); + + // validate partition columns + let partition_columns = if let Some(active_part) = active_partitions { + if let Some(ref partition_columns) = this.partition_columns { + if &active_part != partition_columns { + Err(WriteError::PartitionColumnMismatch { + expected: active_part, + got: partition_columns.to_vec(), + }) + } else { + Ok(partition_columns.clone()) + } + } else { + Ok(active_part) + } + } else { + Ok(this.partition_columns.unwrap_or_default()) + }?; + let mut schema_drift = false; + let plan = if let Some(plan) = this.input { + if this.schema_mode == Some(SchemaMode::Merge) { + return Err(DeltaTableError::Generic( + "Schema merge not supported yet for Datafusion".to_string(), + )); + } + Ok(plan) + } else if let Some(batches) = this.batches { + if batches.is_empty() { + Err(WriteError::MissingData) + } else { + let schema = batches[0].schema(); + + let mut new_schema = None; + if let Some(snapshot) = &this.snapshot { + let table_schema = snapshot + .physical_arrow_schema(this.log_store.object_store().clone()) + .await + .or_else(|_| snapshot.arrow_schema()) + .unwrap_or(schema.clone()); + + if let Err(schema_err) = + try_cast_batch(schema.fields(), table_schema.fields()) + { + schema_drift = true; + if this.mode == SaveMode::Overwrite && this.schema_mode.is_some() { + new_schema = None // we overwrite anyway, so no need to cast + } else if this.schema_mode == Some(SchemaMode::Merge) { + new_schema = Some(Arc::new(merge_schema( + table_schema.as_ref().clone(), + schema.as_ref().clone(), + )?)); + } else { + return Err(schema_err.into()); + } + } + } + + let data = if !partition_columns.is_empty() { + // TODO partitioning should probably happen in its own plan ... + let mut partitions: HashMap> = HashMap::new(); + for batch in batches { + let real_batch = match new_schema.clone() { + Some(new_schema) => { + cast_record_batch(&batch, new_schema, false, true)? + } + None => batch, + }; + + let divided = divide_by_partition_values( + new_schema.clone().unwrap_or(schema.clone()), + partition_columns.clone(), + &real_batch, + )?; + for part in divided { + let key = part.partition_values.hive_partition_path(); + match partitions.get_mut(&key) { + Some(part_batches) => { + part_batches.push(part.record_batch); + } + None => { + partitions.insert(key, vec![part.record_batch]); + } + } + } + } + partitions.into_values().collect::>() + } else { + match new_schema { + Some(ref new_schema) => { + let mut new_batches = vec![]; + for batch in batches { + new_batches.push(cast_record_batch( + &batch, + new_schema.clone(), + false, + true, + )?); + } + vec![new_batches] + } + None => vec![batches], + } + }; + + Ok(Arc::new(MemoryExec::try_new( + &data, + new_schema.unwrap_or(schema).clone(), + None, + )?) as Arc) + } + } else { + Err(WriteError::MissingData) + }?; + let schema = plan.schema(); + if this.schema_mode == Some(SchemaMode::Merge) && schema_drift { + if let Some(snapshot) = &this.snapshot { + let schema_struct: StructType = schema.clone().try_into()?; + let schema_action = Action::Metadata(Metadata::try_new( + schema_struct, + partition_columns.clone(), + snapshot.metadata().configuration.clone(), + )?); + actions.push(schema_action); + } + } + let state = match this.state { + Some(state) => state, + None => { + let ctx = SessionContext::new(); + register_store(this.log_store.clone(), ctx.runtime_env()); + ctx.state() + } + }; + + let (predicate_str, predicate) = match this.predicate { + Some(predicate) => { + let pred = match predicate { + Expression::DataFusion(expr) => expr, + Expression::String(s) => { + let df_schema = DFSchema::try_from(schema.as_ref().to_owned())?; + parse_predicate_expression(&df_schema, s, &state)? + // this.snapshot.unwrap().parse_predicate_expression(s, &state)? + } + }; + (Some(fmt_expr_to_sql(&pred)?), Some(pred)) + } + _ => (None, None), + }; + + // Here we need to validate if the new data conforms to a predicate if one is provided + let add_actions = write_execution_plan_with_predicate( + predicate.clone(), + this.snapshot.as_ref(), + state.clone(), + plan, + partition_columns.clone(), + this.log_store.object_store().clone(), + this.target_file_size, + this.write_batch_size, + this.writer_properties.clone(), + this.safe_cast, + this.schema_mode, + ) + .await?; + actions.extend(add_actions); + + // Collect remove actions if we are overwriting the table + if let Some(snapshot) = &this.snapshot { + if matches!(this.mode, SaveMode::Overwrite) { + // Update metadata with new schema + let table_schema = snapshot + .physical_arrow_schema(this.log_store.object_store().clone()) + .await + .or_else(|_| snapshot.arrow_schema()) + .unwrap_or(schema.clone()); + + if schema != table_schema { + let mut metadata = snapshot.metadata().clone(); + let delta_schema: StructType = schema.as_ref().try_into()?; + metadata.schema_string = serde_json::to_string(&delta_schema)?; + actions.push(Action::Metadata(metadata)); + } + + let deletion_timestamp = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as i64; + + match predicate { + Some(pred) => { + let predicate_actions = prepare_predicate_actions( + pred, + this.log_store.clone(), + snapshot, + state, + partition_columns.clone(), + this.writer_properties, + deletion_timestamp, + ) + .await?; + if !predicate_actions.is_empty() { + actions.extend(predicate_actions); + } + } + _ => { + let remove_actions = snapshot + .log_data() + .into_iter() + .map(|p| p.remove_action(true).into()); + actions.extend(remove_actions); + } + }; + } + } + + let operation = DeltaOperation::Write { + mode: this.mode, + partition_by: if !partition_columns.is_empty() { + Some(partition_columns) + } else { + None + }, + predicate: predicate_str, + }; + + let commit = CommitBuilder::from(this.commit_properties) + .with_actions(actions) + .build( + this.snapshot.as_ref().map(|f| f as &dyn TableReference), + this.log_store.clone(), + operation.clone(), + )? + .await?; + + // TODO we do not have the table config available, but since we are merging only our newly + // created actions, it may be safe to assume, that we want to include all actions. + // then again, having only some tombstones may be misleading. + if let Some(mut snapshot) = this.snapshot { + snapshot.merge(commit.data.actions, &commit.data.operation, commit.version)?; + Ok(DeltaTable::new_with_state(this.log_store, snapshot)) + } else { + let mut table = DeltaTable::new(this.log_store, Default::default()); + table.update().await?; + Ok(table) + } + }) + } +} + +fn try_cast_batch(from_fields: &Fields, to_fields: &Fields) -> Result<(), ArrowError> { + if from_fields.len() != to_fields.len() { + return Err(ArrowError::SchemaError(format!( + "Cannot cast schema, number of fields does not match: {} vs {}", + from_fields.len(), + to_fields.len() + ))); + } + + from_fields + .iter() + .map(|f| { + if let Some((_, target_field)) = to_fields.find(f.name()) { + if let (DataType::Struct(fields0), DataType::Struct(fields1)) = + (f.data_type(), target_field.data_type()) + { + try_cast_batch(fields0, fields1) + } else if !can_cast_types(f.data_type(), target_field.data_type()) { + Err(ArrowError::SchemaError(format!( + "Cannot cast field {} from {} to {}", + f.name(), + f.data_type(), + target_field.data_type() + ))) + } else { + Ok(()) + } + } else { + Err(ArrowError::SchemaError(format!( + "Field {} not found in schema", + f.name() + ))) + } + }) + .collect::, _>>()?; + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::operations::{collect_sendable_stream, DeltaOps}; + use crate::protocol::SaveMode; + use crate::writer::test_utils::datafusion::write_batch; + use crate::writer::test_utils::datafusion::{get_data, get_data_sorted}; + use crate::writer::test_utils::{ + get_arrow_schema, get_delta_schema, get_delta_schema_with_nested_struct, get_record_batch, + get_record_batch_with_nested_struct, setup_table_with_configuration, + }; + use crate::DeltaConfigKey; + use arrow::datatypes::Field; + use arrow::datatypes::Schema as ArrowSchema; + use arrow_array::{Int32Array, StringArray, TimestampMicrosecondArray}; + use arrow_schema::{DataType, TimeUnit}; + use datafusion::prelude::*; + use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; + use serde_json::{json, Value}; + + #[tokio::test] + async fn test_write_when_delta_table_is_append_only() { + let table = setup_table_with_configuration(DeltaConfigKey::AppendOnly, Some("true")).await; + let batch = get_record_batch(None, false); + // Append + let table = write_batch(table, batch.clone()).await; + // Overwrite + let _err = DeltaOps(table) + .write(vec![batch]) + .with_save_mode(SaveMode::Overwrite) + .await + .expect_err("Remove action is included when Delta table is append-only. Should error"); + } + + #[tokio::test] + async fn test_create_write() { + let table_schema = get_delta_schema(); + let batch = get_record_batch(None, false); + + let table = DeltaOps::new_in_memory() + .create() + .with_columns(table_schema.fields().clone()) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.history(None).await.unwrap().len(), 1); + + // write some data + let metadata = HashMap::from_iter(vec![("k1".to_string(), json!("v1.1"))]); + let mut table = DeltaOps(table) + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::Append) + .with_commit_properties(CommitProperties::default().with_metadata(metadata.clone())) + .await + .unwrap(); + assert_eq!(table.version(), 1); + assert_eq!(table.get_files_count(), 1); + table.load().await.unwrap(); + assert_eq!(table.history(None).await.unwrap().len(), 2); + assert_eq!( + table.history(None).await.unwrap()[0] + .info + .clone() + .into_iter() + .filter(|(k, _)| k != "clientVersion") + .collect::>(), + metadata + ); + + // append some data + let metadata: HashMap = + HashMap::from_iter(vec![("k1".to_string(), json!("v1.2"))]); + let mut table = DeltaOps(table) + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::Append) + .with_commit_properties(CommitProperties::default().with_metadata(metadata.clone())) + .await + .unwrap(); + assert_eq!(table.version(), 2); + assert_eq!(table.get_files_count(), 2); + table.load().await.unwrap(); + assert_eq!(table.history(None).await.unwrap().len(), 3); + assert_eq!( + table.history(None).await.unwrap()[0] + .info + .clone() + .into_iter() + .filter(|(k, _)| k != "clientVersion") + .collect::>(), + metadata + ); + + // overwrite table + let metadata: HashMap = + HashMap::from_iter(vec![("k2".to_string(), json!("v2.1"))]); + let mut table = DeltaOps(table) + .write(vec![batch]) + .with_save_mode(SaveMode::Overwrite) + .with_commit_properties(CommitProperties::default().with_metadata(metadata.clone())) + .await + .unwrap(); + assert_eq!(table.version(), 3); + assert_eq!(table.get_files_count(), 1); + table.load().await.unwrap(); + assert_eq!(table.history(None).await.unwrap().len(), 4); + assert_eq!( + table.history(None).await.unwrap()[0] + .info + .clone() + .into_iter() + .filter(|(k, _)| k != "clientVersion") + .collect::>(), + metadata + ); + } + + #[tokio::test] + async fn test_write_different_types() { + // Ensure write data is casted when data of a different type from the table is provided. + + // Validate String -> Int is err + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "value", + DataType::Int32, + true, + )])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![Some(0), None]))], + ) + .unwrap(); + let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "value", + DataType::Utf8, + true, + )])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(StringArray::from(vec![ + Some("Test123".to_owned()), + Some("123".to_owned()), + None, + ]))], + ) + .unwrap(); + + // Test cast options + let table = DeltaOps::from(table) + .write(vec![batch.clone()]) + .with_cast_safety(true) + .await + .unwrap(); + + let expected = [ + "+-------+", + "| value |", + "+-------+", + "| |", + "| |", + "| |", + "| 123 |", + "| 0 |", + "+-------+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + + let res = DeltaOps::from(table).write(vec![batch]).await; + assert!(res.is_err()); + + // Validate the datetime -> string behavior + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "value", + arrow::datatypes::DataType::Utf8, + true, + )])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(StringArray::from(vec![Some( + "2023-06-03 15:35:00".to_owned(), + )]))], + ) + .unwrap(); + let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "value", + DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())), + true, + )])); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new( + TimestampMicrosecondArray::from(vec![Some(10000)]).with_timezone("UTC"), + )], + ) + .unwrap(); + + let _res = DeltaOps::from(table).write(vec![batch]).await.unwrap(); + let expected = [ + "+--------------------------+", + "| value |", + "+--------------------------+", + "| 1970-01-01T00:00:00.010Z |", + "| 2023-06-03 15:35:00 |", + "+--------------------------+", + ]; + let actual = get_data(&_res).await; + assert_batches_sorted_eq!(&expected, &actual); + } + + #[tokio::test] + async fn test_write_nonexistent() { + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_save_mode(SaveMode::ErrorIfExists) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_files_count(), 1) + } + + #[tokio::test] + async fn test_write_partitioned() { + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::ErrorIfExists) + .with_partition_columns(["modified"]) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_files_count(), 2); + + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_save_mode(SaveMode::ErrorIfExists) + .with_partition_columns(["modified", "id"]) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_files_count(), 4) + } + + #[tokio::test] + async fn test_merge_schema() { + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::ErrorIfExists) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let mut new_schema_builder = arrow_schema::SchemaBuilder::new(); + for field in batch.schema().fields() { + if field.name() != "modified" { + new_schema_builder.push(field.clone()); + } + } + new_schema_builder.push(Field::new("inserted_by", DataType::Utf8, true)); + let new_schema = new_schema_builder.finish(); + let new_fields = new_schema.fields(); + let new_names = new_fields.iter().map(|f| f.name()).collect::>(); + assert_eq!(new_names, vec!["id", "value", "inserted_by"]); + let inserted_by = StringArray::from(vec![ + Some("A1"), + Some("B1"), + None, + Some("B2"), + Some("A3"), + Some("A4"), + None, + None, + Some("B4"), + Some("A5"), + Some("A7"), + ]); + let new_batch = RecordBatch::try_new( + Arc::new(new_schema), + vec![ + Arc::new(batch.column_by_name("id").unwrap().clone()), + Arc::new(batch.column_by_name("value").unwrap().clone()), + Arc::new(inserted_by), + ], + ) + .unwrap(); + + let mut table = DeltaOps(table) + .write(vec![new_batch]) + .with_save_mode(SaveMode::Append) + .with_schema_mode(SchemaMode::Merge) + .await + .unwrap(); + table.load().await.unwrap(); + assert_eq!(table.version(), 1); + let new_schema = table.metadata().unwrap().schema().unwrap(); + let fields = new_schema.fields(); + let names = fields.iter().map(|f| f.name()).collect::>(); + assert_eq!(names, vec!["id", "value", "modified", "inserted_by"]); + } + + #[tokio::test] + async fn test_merge_schema_with_partitions() { + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_partition_columns(vec!["id", "value"]) + .with_save_mode(SaveMode::ErrorIfExists) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let mut new_schema_builder = arrow_schema::SchemaBuilder::new(); + for field in batch.schema().fields() { + if field.name() != "modified" { + new_schema_builder.push(field.clone()); + } + } + new_schema_builder.push(Field::new("inserted_by", DataType::Utf8, true)); + let new_schema = new_schema_builder.finish(); + let new_fields = new_schema.fields(); + let new_names = new_fields.iter().map(|f| f.name()).collect::>(); + assert_eq!(new_names, vec!["id", "value", "inserted_by"]); + let inserted_by = StringArray::from(vec![ + Some("A1"), + Some("B1"), + None, + Some("B2"), + Some("A3"), + Some("A4"), + None, + None, + Some("B4"), + Some("A5"), + Some("A7"), + ]); + let new_batch = RecordBatch::try_new( + Arc::new(new_schema), + vec![ + Arc::new(batch.column_by_name("id").unwrap().clone()), + Arc::new(batch.column_by_name("value").unwrap().clone()), + Arc::new(inserted_by), + ], + ) + .unwrap(); + println!("new_batch: {:?}", new_batch.schema()); + let table = DeltaOps(table) + .write(vec![new_batch]) + .with_save_mode(SaveMode::Append) + .with_schema_mode(SchemaMode::Merge) + .await + .unwrap(); + + assert_eq!(table.version(), 1); + let new_schema = table.metadata().unwrap().schema().unwrap(); + let fields = new_schema.fields(); + let mut names = fields.iter().map(|f| f.name()).collect::>(); + names.sort(); + assert_eq!(names, vec!["id", "inserted_by", "modified", "value"]); + let part_cols = table.metadata().unwrap().partition_columns.clone(); + assert_eq!(part_cols, vec!["id", "value"]); // we want to preserve partitions + } + + #[tokio::test] + async fn test_overwrite_schema() { + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::ErrorIfExists) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let mut new_schema_builder = arrow_schema::SchemaBuilder::new(); + for field in batch.schema().fields() { + if field.name() != "modified" { + new_schema_builder.push(field.clone()); + } + } + new_schema_builder.push(Field::new("inserted_by", DataType::Utf8, true)); + let new_schema = new_schema_builder.finish(); + let new_fields = new_schema.fields(); + let new_names = new_fields.iter().map(|f| f.name()).collect::>(); + assert_eq!(new_names, vec!["id", "value", "inserted_by"]); + let inserted_by = StringArray::from(vec![ + Some("A1"), + Some("B1"), + None, + Some("B2"), + Some("A3"), + Some("A4"), + None, + None, + Some("B4"), + Some("A5"), + Some("A7"), + ]); + let new_batch = RecordBatch::try_new( + Arc::new(new_schema), + vec![ + Arc::new(batch.column_by_name("id").unwrap().clone()), + Arc::new(batch.column_by_name("value").unwrap().clone()), + Arc::new(inserted_by), + ], + ) + .unwrap(); + + let table = DeltaOps(table) + .write(vec![new_batch]) + .with_save_mode(SaveMode::Append) + .with_schema_mode(SchemaMode::Overwrite) + .await; + assert!(table.is_err()); + } + + #[tokio::test] + async fn test_overwrite_check() { + // If you do not pass a schema mode, we want to check the schema + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::ErrorIfExists) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let mut new_schema_builder = arrow_schema::SchemaBuilder::new(); + + new_schema_builder.push(Field::new("inserted_by", DataType::Utf8, true)); + let new_schema = new_schema_builder.finish(); + let new_fields = new_schema.fields(); + let new_names = new_fields.iter().map(|f| f.name()).collect::>(); + assert_eq!(new_names, vec!["inserted_by"]); + let inserted_by = StringArray::from(vec![ + Some("A1"), + Some("B1"), + None, + Some("B2"), + Some("A3"), + Some("A4"), + None, + None, + Some("B4"), + Some("A5"), + Some("A7"), + ]); + let new_batch = + RecordBatch::try_new(Arc::new(new_schema), vec![Arc::new(inserted_by)]).unwrap(); + + let table = DeltaOps(table) + .write(vec![new_batch]) + .with_save_mode(SaveMode::Append) + .await; + assert!(table.is_err()); + } + + #[tokio::test] + async fn test_check_invariants() { + let batch = get_record_batch(None, false); + let schema: StructType = serde_json::from_value(json!({ + "type": "struct", + "fields": [ + {"name": "id", "type": "string", "nullable": true, "metadata": {}}, + {"name": "value", "type": "integer", "nullable": true, "metadata": { + "delta.invariants": "{\"expression\": { \"expression\": \"value < 12\"} }" + }}, + {"name": "modified", "type": "string", "nullable": true, "metadata": {}}, + ] + })) + .unwrap(); + let table = DeltaOps::new_in_memory() + .create() + .with_save_mode(SaveMode::ErrorIfExists) + .with_columns(schema.fields().clone()) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let table = DeltaOps(table).write(vec![batch.clone()]).await.unwrap(); + assert_eq!(table.version(), 1); + + let schema: StructType = serde_json::from_value(json!({ + "type": "struct", + "fields": [ + {"name": "id", "type": "string", "nullable": true, "metadata": {}}, + {"name": "value", "type": "integer", "nullable": true, "metadata": { + "delta.invariants": "{\"expression\": { \"expression\": \"value < 6\"} }" + }}, + {"name": "modified", "type": "string", "nullable": true, "metadata": {}}, + ] + })) + .unwrap(); + let table = DeltaOps::new_in_memory() + .create() + .with_save_mode(SaveMode::ErrorIfExists) + .with_columns(schema.fields().clone()) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let table = DeltaOps(table).write(vec![batch.clone()]).await; + assert!(table.is_err()) + } + + #[tokio::test] + async fn test_nested_struct() { + let table_schema = get_delta_schema_with_nested_struct(); + let batch = get_record_batch_with_nested_struct(); + + let table = DeltaOps::new_in_memory() + .create() + .with_columns(table_schema.fields().clone()) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let table = DeltaOps(table) + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::Append) + .await + .unwrap(); + assert_eq!(table.version(), 1); + + let actual = get_data(&table).await; + let expected = DataType::Struct(Fields::from(vec![Field::new( + "count", + DataType::Int32, + true, + )])); + assert_eq!( + actual[0].column_by_name("nested").unwrap().data_type(), + &expected + ); + } + + #[tokio::test] + async fn test_special_characters_write_read() { + let tmp_dir = tempfile::tempdir().unwrap(); + let tmp_path = std::fs::canonicalize(tmp_dir.path()).unwrap(); + + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("string", DataType::Utf8, true), + Field::new("data", DataType::Utf8, true), + ])); + + let str_values = StringArray::from(vec![r#"$%&/()=^"[]#*?._- {=}|`<>~/\r\n+"#]); + let data_values = StringArray::from(vec!["test"]); + + let batch = RecordBatch::try_new(schema, vec![Arc::new(str_values), Arc::new(data_values)]) + .unwrap(); + + let ops = DeltaOps::try_from_uri(tmp_path.as_os_str().to_str().unwrap()) + .await + .unwrap(); + + let _table = ops + .write([batch.clone()]) + .with_partition_columns(["string"]) + .await + .unwrap(); + + let table = crate::open_table(tmp_path.as_os_str().to_str().unwrap()) + .await + .unwrap(); + let (_table, stream) = DeltaOps(table).load().await.unwrap(); + let data: Vec = collect_sendable_stream(stream).await.unwrap(); + + let expected = vec![ + "+------+----------------------------------+", + "| data | string |", + "+------+----------------------------------+", + "| test | $%&/()=^\"[]#*?._- {=}|`<>~/\\r\\n+ |", + "+------+----------------------------------+", + ]; + + assert_batches_eq!(&expected, &data); + } + + #[tokio::test] + async fn test_replace_where() { + let schema = get_arrow_schema(&None); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "C", "C"])), + Arc::new(arrow::array::Int32Array::from(vec![0, 20, 10, 100])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2021-02-03", + "2021-02-02", + "2021-02-04", + ])), + ], + ) + .unwrap(); + + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_save_mode(SaveMode::Append) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let batch_add = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["C"])), + Arc::new(arrow::array::Int32Array::from(vec![50])), + Arc::new(arrow::array::StringArray::from(vec!["2023-01-01"])), + ], + ) + .unwrap(); + + let table = DeltaOps(table) + .write(vec![batch_add]) + .with_save_mode(SaveMode::Overwrite) + .with_replace_where(col("id").eq(lit("C"))) + .await + .unwrap(); + assert_eq!(table.version(), 1); + + let expected = [ + "+----+-------+------------+", + "| id | value | modified |", + "+----+-------+------------+", + "| A | 0 | 2021-02-02 |", + "| B | 20 | 2021-02-03 |", + "| C | 50 | 2023-01-01 |", + "+----+-------+------------+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } + + #[tokio::test] + async fn test_replace_where_fail_not_matching_predicate() { + let schema = get_arrow_schema(&None); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "C", "C"])), + Arc::new(arrow::array::Int32Array::from(vec![0, 20, 10, 100])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2021-02-03", + "2021-02-02", + "2021-02-04", + ])), + ], + ) + .unwrap(); + + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_save_mode(SaveMode::Append) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + // Take clones of these before an operation resulting in error, otherwise it will + // be impossible to refer to an in-memory table + let table_logstore = table.log_store.clone(); + let table_state = table.state.clone().unwrap(); + + // An attempt to write records non comforming to predicate should fail + let batch_fail = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["D"])), + Arc::new(arrow::array::Int32Array::from(vec![1000])), + Arc::new(arrow::array::StringArray::from(vec!["2023-01-01"])), + ], + ) + .unwrap(); + + let table = DeltaOps(table) + .write(vec![batch_fail]) + .with_save_mode(SaveMode::Overwrite) + .with_replace_where(col("id").eq(lit("C"))) + .await; + assert!(table.is_err()); + + // Verify that table state hasn't changed + let table = DeltaTable::new_with_state(table_logstore, table_state); + assert_eq!(table.get_latest_version().await.unwrap(), 0); + } + + #[tokio::test] + async fn test_replace_where_partitioned() { + let schema = get_arrow_schema(&None); + + let batch = get_record_batch(None, false); + + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_partition_columns(["id", "value"]) + .with_save_mode(SaveMode::Append) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + let batch_add = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["A", "A", "A"])), + Arc::new(arrow::array::Int32Array::from(vec![11, 13, 15])), + Arc::new(arrow::array::StringArray::from(vec![ + "2024-02-02", + "2024-02-02", + "2024-02-01", + ])), + ], + ) + .unwrap(); + + let table = DeltaOps(table) + .write(vec![batch_add]) + .with_save_mode(SaveMode::Overwrite) + .with_replace_where(col("id").eq(lit("A"))) + .await + .unwrap(); + assert_eq!(table.version(), 1); + + let expected = [ + "+----+-------+------------+", + "| id | value | modified |", + "+----+-------+------------+", + "| A | 11 | 2024-02-02 |", + "| A | 13 | 2024-02-02 |", + "| A | 15 | 2024-02-01 |", + "| B | 2 | 2021-02-02 |", + "| B | 4 | 2021-02-01 |", + "| B | 8 | 2021-02-01 |", + "| B | 9 | 2021-02-01 |", + "+----+-------+------------+", + ]; + let actual = get_data_sorted(&table, "id,value,modified").await; + assert_batches_sorted_eq!(&expected, &actual); + } +} diff --git a/crates/deltalake-core/src/operations/writer.rs b/crates/core/src/operations/writer.rs similarity index 72% rename from crates/deltalake-core/src/operations/writer.rs rename to crates/core/src/operations/writer.rs index 6d551ecb96..c778ddfad5 100644 --- a/crates/deltalake-core/src/operations/writer.rs +++ b/crates/core/src/operations/writer.rs @@ -6,20 +6,22 @@ use arrow::datatypes::SchemaRef as ArrowSchemaRef; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use bytes::Bytes; +use indexmap::IndexMap; use object_store::{path::Path, ObjectStore}; use parquet::arrow::ArrowWriter; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; +use tracing::debug; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::Add; +use crate::kernel::{Add, PartitionsExt, Scalar}; use crate::storage::ObjectStoreRef; use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; use crate::writer::stats::create_add; use crate::writer::utils::{ arrow_schema_without_partitions, next_data_path, record_batch_without_partitions, - PartitionPath, ShareableBuffer, + ShareableBuffer, }; // TODO databricks often suggests a file size of 100mb, should we set this default? @@ -39,11 +41,6 @@ enum WriteError { source: Box, }, - #[error("Error creating file name from partition info: {source}")] - FileName { - source: Box, - }, - #[error("Error handling Arrow data: {source}")] Arrow { #[from] @@ -69,6 +66,7 @@ impl From for DeltaTableError { } /// Configuration to write data into Delta tables +#[derive(Debug)] pub struct WriterConfig { /// Schema of the delta table table_schema: ArrowSchemaRef, @@ -115,6 +113,7 @@ impl WriterConfig { } } +#[derive(Debug)] /// A parquet writer implementation tailored to the needs of writing data to a delta table. pub struct DeltaWriter { /// An object store pointing at Delta table root @@ -159,12 +158,9 @@ impl DeltaWriter { pub async fn write_partition( &mut self, record_batch: RecordBatch, - partition_values: &HashMap>, + partition_values: &IndexMap, ) -> DeltaResult<()> { - let partition_key = - PartitionPath::from_hashmap(&self.config.partition_columns, partition_values)? - .as_ref() - .into(); + let partition_key = Path::parse(partition_values.hive_partition_path())?; let record_batch = record_batch_without_partitions(&record_batch, &self.config.partition_columns)?; @@ -177,7 +173,6 @@ impl DeltaWriter { let config = PartitionWriterConfig::try_new( self.config.file_schema(), partition_values.clone(), - self.config.partition_columns.clone(), Some(self.config.writer_properties.clone()), Some(self.config.target_file_size), Some(self.config.write_batch_size), @@ -219,13 +214,14 @@ impl DeltaWriter { } } +#[derive(Debug)] pub(crate) struct PartitionWriterConfig { /// Schema of the data written to disk file_schema: ArrowSchemaRef, /// Prefix applied to all paths prefix: Path, /// Values for all partition columns - partition_values: HashMap>, + partition_values: IndexMap, /// Properties passed to underlying parquet writer writer_properties: WriterProperties, /// Size above which we will write a buffered parquet file to disk. @@ -238,17 +234,13 @@ pub(crate) struct PartitionWriterConfig { impl PartitionWriterConfig { pub fn try_new( file_schema: ArrowSchemaRef, - partition_values: HashMap>, - partition_columns: Vec, + partition_values: IndexMap, writer_properties: Option, target_file_size: Option, write_batch_size: Option, ) -> DeltaResult { - let part_path = PartitionPath::from_hashmap(&partition_columns, &partition_values) - .map_err(|err| WriteError::FileName { - source: Box::new(err), - })?; - let prefix = Path::parse(part_path.as_ref())?; + let part_path = partition_values.hive_partition_path(); + let prefix = Path::parse(part_path)?; let writer_properties = writer_properties.unwrap_or_else(|| { WriterProperties::builder() .set_created_by(format!("delta-rs version {}", crate_version())) @@ -268,6 +260,7 @@ impl PartitionWriterConfig { } } +#[derive(Debug)] pub(crate) struct PartitionWriter { object_store: ObjectStoreRef, writer_id: uuid::Uuid, @@ -334,6 +327,11 @@ impl PartitionWriter { // replace counter / buffers and close the current writer let (writer, buffer) = self.reset_writer()?; let metadata = writer.close()?; + // don't write empty file + if metadata.num_rows == 0 { + return Ok(()); + } + let buffer = match buffer.into_inner() { Some(buffer) => Bytes::from(buffer), None => return Ok(()), // Nothing to write @@ -379,8 +377,12 @@ impl PartitionWriter { let length = usize::min(self.config.write_batch_size, max_offset - offset); self.write_batch(&batch.slice(offset, length))?; // flush currently buffered data to disk once we meet or exceed the target file size. - if self.buffer.len() >= self.config.target_file_size { - log::debug!("Writing file with size {:?} to disk.", self.buffer.len()); + let estimated_size = self.buffer.len() + self.arrow_writer.in_progress_size(); + if estimated_size >= self.config.target_file_size { + debug!( + "Writing file with estimated size {:?} to disk.", + estimated_size + ); self.flush_arrow_writer().await?; } } @@ -398,12 +400,47 @@ impl PartitionWriter { mod tests { use super::*; use crate::storage::utils::flatten_list_stream as list; - use crate::writer::test_utils::get_record_batch; + use crate::writer::test_utils::*; use crate::DeltaTableBuilder; use arrow::array::{Int32Array, StringArray}; use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; use std::sync::Arc; + fn get_delta_writer( + object_store: ObjectStoreRef, + batch: &RecordBatch, + writer_properties: Option, + target_file_size: Option, + write_batch_size: Option, + ) -> DeltaWriter { + let config = WriterConfig::new( + batch.schema(), + vec![], + writer_properties, + target_file_size, + write_batch_size, + ); + DeltaWriter::new(object_store, config) + } + + fn get_partition_writer( + object_store: ObjectStoreRef, + batch: &RecordBatch, + writer_properties: Option, + target_file_size: Option, + write_batch_size: Option, + ) -> PartitionWriter { + let config = PartitionWriterConfig::try_new( + batch.schema(), + IndexMap::new(), + writer_properties, + target_file_size, + write_batch_size, + ) + .unwrap(); + PartitionWriter::try_with_config(object_store, config).unwrap() + } + #[tokio::test] async fn test_write_partition() { let log_store = DeltaTableBuilder::from_uri("memory://") @@ -413,7 +450,7 @@ mod tests { let batch = get_record_batch(None, false); // write single un-partitioned batch - let mut writer = get_writer(object_store.clone(), &batch, None, None); + let mut writer = get_partition_writer(object_store.clone(), &batch, None, None, None); writer.write(&batch).await.unwrap(); let files = list(object_store.as_ref(), None).await.unwrap(); assert_eq!(files.len(), 0); @@ -446,7 +483,8 @@ mod tests { .set_max_row_group_size(1024) .build(); // configure small target file size and and row group size so we can observe multiple files written - let mut writer = get_writer(object_store, &batch, Some(properties), Some(10_000)); + let mut writer = + get_partition_writer(object_store, &batch, Some(properties), Some(10_000), None); writer.write(&batch).await.unwrap(); // check that we have written more then once file, and no more then 1 is below target size @@ -458,21 +496,102 @@ mod tests { assert!(target_file_count >= adds.len() as i32 - 1) } - fn get_writer( - object_store: ObjectStoreRef, - batch: &RecordBatch, - writer_properties: Option, - target_file_size: Option, - ) -> PartitionWriter { - let config = PartitionWriterConfig::try_new( - batch.schema(), - HashMap::new(), - Vec::new(), - writer_properties, - target_file_size, - None, + #[tokio::test] + async fn test_unflushed_row_group_size() { + let base_int = Arc::new(Int32Array::from((0..10000).collect::>())); + let base_str = Arc::new(StringArray::from(vec!["A"; 10000])); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Utf8, true), + Field::new("value", DataType::Int32, true), + ])); + let batch = RecordBatch::try_new(schema, vec![base_str, base_int]).unwrap(); + + let object_store = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap() + .object_store(); + // configure small target file size so we can observe multiple files written + let mut writer = get_partition_writer(object_store, &batch, None, Some(10_000), None); + writer.write(&batch).await.unwrap(); + + // check that we have written more then once file, and no more then 1 is below target size + let adds = writer.close().await.unwrap(); + assert!(adds.len() > 1); + let target_file_count = adds + .iter() + .fold(0, |acc, add| acc + (add.size > 10_000) as i32); + assert!(target_file_count >= adds.len() as i32 - 1) + } + + #[tokio::test] + async fn test_do_not_write_empty_file_on_close() { + let base_int = Arc::new(Int32Array::from((0..10000_i32).collect::>())); + let base_str = Arc::new(StringArray::from(vec!["A"; 10000])); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Utf8, true), + Field::new("value", DataType::Int32, true), + ])); + let batch = RecordBatch::try_new(schema, vec![base_str, base_int]).unwrap(); + + let object_store = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap() + .object_store(); + // configure high batch size and low file size to observe one file written and flushed immediately + // upon writing batch, then ensures the buffer is empty upon closing writer + let mut writer = get_partition_writer(object_store, &batch, None, Some(9000), Some(10000)); + writer.write(&batch).await.unwrap(); + + let adds = writer.close().await.unwrap(); + assert!(adds.len() == 1); + } + + #[tokio::test] + async fn test_write_mismatched_schema() { + let log_store = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap(); + let object_store = log_store.object_store(); + let batch = get_record_batch(None, false); + + // write single un-partitioned batch + let mut writer = get_delta_writer(object_store.clone(), &batch, None, None, None); + writer.write(&batch).await.unwrap(); + // Ensure the write hasn't been flushed + let files = list(object_store.as_ref(), None).await.unwrap(); + assert_eq!(files.len(), 0); + + // Create a second batch with a different schema + let second_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ])); + let second_batch = RecordBatch::try_new( + second_schema, + vec![ + Arc::new(Int32Array::from(vec![Some(1), Some(2)])), + Arc::new(StringArray::from(vec![Some("will"), Some("robert")])), + ], ) .unwrap(); - PartitionWriter::try_with_config(object_store, config).unwrap() + + let result = writer.write(&second_batch).await; + assert!(result.is_err()); + + match result { + Ok(_) => { + assert!(false, "Should not have successfully written"); + } + Err(e) => { + match e { + DeltaTableError::SchemaMismatch { .. } => { + // this is expected + } + others => { + assert!(false, "Got the wrong error: {others:?}"); + } + } + } + }; } } diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs similarity index 74% rename from crates/deltalake-core/src/protocol/checkpoints.rs rename to crates/core/src/protocol/checkpoints.rs index 55b36a64e1..ae6d1debd8 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -1,31 +1,29 @@ //! Implementation for writing delta checkpoints. use std::collections::HashMap; -use std::convert::TryFrom; use std::iter::Iterator; -use arrow::json::ReaderBuilder; -use arrow_schema::{ArrowError, Schema as ArrowSchema}; +use arrow_json::ReaderBuilder; +use arrow_schema::ArrowError; use chrono::{Datelike, Utc}; use futures::{StreamExt, TryStreamExt}; use lazy_static::lazy_static; -use log::*; -use object_store::ObjectStore; +use object_store::{Error, ObjectStore}; use parquet::arrow::ArrowWriter; use parquet::errors::ParquetError; use regex::Regex; use serde_json::Value; +use tracing::{debug, error}; use super::{time_utils, ProtocolError}; use crate::kernel::arrow::delta_log_schema_for_table; use crate::kernel::{ - Action, Add as AddAction, DataType, Metadata, PrimitiveType, Protocol, StructField, StructType, - Txn, + Action, Add as AddAction, DataType, PrimitiveType, Protocol, Remove, StructField, Txn, }; use crate::logstore::LogStore; use crate::table::state::DeltaTableState; -use crate::table::{CheckPoint, CheckPointBuilder}; +use crate::table::{get_partition_col_data_types, CheckPoint, CheckPointBuilder}; use crate::{open_table_with_version, DeltaTable}; type SchemaPath = Vec; @@ -70,12 +68,24 @@ impl From for ProtocolError { } } +use core::str::Utf8Error; +impl From for ProtocolError { + fn from(value: Utf8Error) -> Self { + Self::Generic(value.to_string()) + } +} + /// The record batch size for checkpoint parquet file pub const CHECKPOINT_RECORD_BATCH_SIZE: usize = 5000; /// Creates checkpoint at current table version pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), ProtocolError> { - create_checkpoint_for(table.version(), table.get_state(), table.log_store.as_ref()).await?; + create_checkpoint_for( + table.version(), + table.snapshot().map_err(|_| ProtocolError::NoMetaData)?, + table.log_store.as_ref(), + ) + .await?; Ok(()) } @@ -84,7 +94,8 @@ pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), ProtocolError> pub async fn cleanup_metadata(table: &DeltaTable) -> Result { let log_retention_timestamp = Utc::now().timestamp_millis() - table - .get_state() + .snapshot() + .map_err(|_| ProtocolError::NoMetaData)? .table_config() .log_retention_duration() .as_millis() as i64; @@ -107,14 +118,11 @@ pub async fn create_checkpoint_from_table_uri_and_cleanup( let table = open_table_with_version(table_uri, version) .await .map_err(|err| ProtocolError::Generic(err.to_string()))?; - create_checkpoint_for(version, table.get_state(), table.log_store.as_ref()).await?; + let snapshot = table.snapshot().map_err(|_| ProtocolError::NoMetaData)?; + create_checkpoint_for(version, snapshot, table.log_store.as_ref()).await?; - let enable_expired_log_cleanup = cleanup.unwrap_or_else(|| { - table - .get_state() - .table_config() - .enable_expired_log_cleanup() - }); + let enable_expired_log_cleanup = + cleanup.unwrap_or_else(|| snapshot.table_config().enable_expired_log_cleanup()); if table.version() >= 0 && enable_expired_log_cleanup { let deleted_log_num = cleanup_metadata(&table).await?; @@ -144,7 +152,12 @@ pub async fn create_checkpoint_for( let last_checkpoint_path = log_store.log_path().child("_last_checkpoint"); debug!("Writing parquet bytes to checkpoint buffer."); - let (checkpoint, parquet_bytes) = parquet_bytes_from_state(state)?; + let tombstones = state + .unexpired_tombstones(log_store.object_store().clone()) + .await + .map_err(|_| ProtocolError::Generic("filed to get tombstones".into()))? + .collect::>(); + let (checkpoint, parquet_bytes) = parquet_bytes_from_state(state, tombstones)?; let file_name = format!("{version:020}.checkpoint.parquet"); let checkpoint_path = log_store.log_path().child(file_name); @@ -176,6 +189,19 @@ pub async fn cleanup_expired_logs_for( Regex::new(r"_delta_log/(\d{20})\.(json|checkpoint).*$").unwrap(); } + let object_store = log_store.object_store(); + let maybe_last_checkpoint = object_store + .get(&log_store.log_path().child("_last_checkpoint")) + .await; + + if let Err(Error::NotFound { path: _, source: _ }) = maybe_last_checkpoint { + return Ok(0); + } + + let last_checkpoint = maybe_last_checkpoint?.bytes().await?; + let last_checkpoint: CheckPoint = serde_json::from_slice(&last_checkpoint)?; + let until_version = i64::min(until_version, last_checkpoint.version); + // Feed a stream of candidate deletion files directly into the delete_stream // function to try to improve the speed of cleanup and reduce the need for // intermediate memory. @@ -184,7 +210,6 @@ pub async fn cleanup_expired_logs_for( .delete_stream( object_store .list(Some(log_store.log_path())) - .await? // This predicate function will filter out any locations that don't // match the given timestamp range .filter_map(|meta: Result| async move { @@ -220,19 +245,16 @@ pub async fn cleanup_expired_logs_for( fn parquet_bytes_from_state( state: &DeltaTableState, + mut tombstones: Vec, ) -> Result<(CheckPoint, bytes::Bytes), ProtocolError> { - let current_metadata = state.metadata().ok_or(ProtocolError::NoMetaData)?; + let current_metadata = state.metadata(); + let schema = current_metadata.schema()?; - let partition_col_data_types = current_metadata.get_partition_col_data_types(); + let partition_col_data_types = get_partition_col_data_types(&schema, current_metadata); // Collect a map of paths that require special stats conversion. let mut stats_conversions: Vec<(SchemaPath, DataType)> = Vec::new(); - collect_stats_conversions( - &mut stats_conversions, - current_metadata.schema.fields().as_slice(), - ); - - let mut tombstones = state.unexpired_tombstones().cloned().collect::>(); + collect_stats_conversions(&mut stats_conversions, schema.fields().as_slice()); // if any, tombstones do not include extended file metadata, we must omit the extended metadata fields from the remove schema // See https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file @@ -252,7 +274,7 @@ fn parquet_bytes_from_state( remove.extended_file_metadata = Some(false); } } - + let files = state.file_actions().unwrap(); // protocol let jsons = std::iter::once(Action::Protocol(Protocol { min_reader_version: state.protocol().min_reader_version, @@ -261,9 +283,7 @@ fn parquet_bytes_from_state( reader_features: None, })) // metaData - .chain(std::iter::once(Action::Metadata(Metadata::try_from( - current_metadata.clone(), - )?))) + .chain(std::iter::once(Action::Metadata(current_metadata.clone()))) // txns .chain( state @@ -291,13 +311,13 @@ fn parquet_bytes_from_state( })) .map(|a| serde_json::to_value(a).map_err(ProtocolError::from)) // adds - .chain(state.files().iter().map(|f| { + .chain(files.iter().map(|f| { checkpoint_add_from_state(f, partition_col_data_types.as_slice(), &stats_conversions) })); // Create the arrow schema that represents the Checkpoint parquet file. let arrow_schema = delta_log_schema_for_table( - >::try_from(¤t_metadata.schema)?, + (&schema).try_into()?, current_metadata.partition_columns.as_slice(), use_extended_remove_schema, ); @@ -500,13 +520,22 @@ fn apply_stats_conversion( #[cfg(test)] mod tests { - use super::*; + use std::sync::Arc; + + use arrow_array::builder::{Int32Builder, ListBuilder, StructBuilder}; + use arrow_array::{ArrayRef, Int32Array, RecordBatch}; + use arrow_schema::Schema as ArrowSchema; + use chrono::Duration; use lazy_static::lazy_static; + use object_store::path::Path; use serde_json::json; + use super::*; + use crate::kernel::StructType; + use crate::operations::transaction::{CommitBuilder, TableReference}; use crate::operations::DeltaOps; + use crate::protocol::Metadata; use crate::writer::test_utils::get_delta_schema; - use object_store::path::Path; #[tokio::test] async fn test_create_checkpoint_for() { @@ -520,7 +549,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); assert_eq!(table.get_schema().unwrap(), &table_schema); - let res = create_checkpoint_for(0, table.get_state(), table.log_store.as_ref()).await; + let res = + create_checkpoint_for(0, table.snapshot().unwrap(), table.log_store.as_ref()).await; assert!(res.is_ok()); // Look at the "files" and verify that the _last_checkpoint has the right version @@ -537,6 +567,85 @@ mod tests { assert_eq!(last_checkpoint.version, 0); } + /// This test validates that a checkpoint can be written and re-read with the minimum viable + /// Metadata. There was a bug which didn't handle the optionality of createdTime. + #[tokio::test] + async fn test_create_checkpoint_with_metadata() { + let table_schema = get_delta_schema(); + + let mut table = DeltaOps::new_in_memory() + .create() + .with_columns(table_schema.fields().clone()) + .with_save_mode(crate::protocol::SaveMode::Ignore) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_schema().unwrap(), &table_schema); + + let part_cols: Vec = vec![]; + let metadata = Metadata::try_new(table_schema, part_cols, HashMap::new()).unwrap(); + let actions = vec![Action::Metadata(metadata)]; + + let epoch_id = std::time::SystemTime::now() + .duration_since(std::time::UNIX_EPOCH) + .expect("Time went backwards") + .as_millis() as i64; + + let operation = crate::protocol::DeltaOperation::StreamingUpdate { + output_mode: crate::protocol::OutputMode::Append, + query_id: "test".into(), + epoch_id, + }; + let v = CommitBuilder::default() + .with_actions(actions) + .build( + table.state.as_ref().map(|f| f as &dyn TableReference), + table.log_store(), + operation, + ) + .unwrap() + .await + .unwrap() + .version(); + + assert_eq!(1, v, "Expected the commit to create table version 1"); + table.load().await.expect("Failed to reload table"); + assert_eq!( + table.version(), + 1, + "The loaded version of the table is not up to date" + ); + + let res = create_checkpoint_for( + table.version(), + table.state.as_ref().unwrap(), + table.log_store.as_ref(), + ) + .await; + assert!(res.is_ok()); + + // Look at the "files" and verify that the _last_checkpoint has the right version + let path = Path::from("_delta_log/_last_checkpoint"); + let last_checkpoint = table + .object_store() + .get(&path) + .await + .expect("Failed to get the _last_checkpoint") + .bytes() + .await + .expect("Failed to get bytes for _last_checkpoint"); + let last_checkpoint: CheckPoint = serde_json::from_slice(&last_checkpoint).expect("Fail"); + assert_eq!(last_checkpoint.version, 1); + + // If the regression exists, this will fail + table.load().await.expect("Failed to reload the table, this likely means that the optional createdTime was not actually optional"); + assert_eq!( + 1, + table.version(), + "The reloaded table doesn't have the right version" + ); + } + #[tokio::test] async fn test_create_checkpoint_for_invalid_version() { let table_schema = get_delta_schema(); @@ -549,7 +658,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); assert_eq!(table.get_schema().unwrap(), &table_schema); - match create_checkpoint_for(1, table.get_state(), table.log_store.as_ref()).await { + match create_checkpoint_for(1, table.snapshot().unwrap(), table.log_store.as_ref()).await { Ok(_) => { /* * If a checkpoint is allowed to be created here, it will use the passed in @@ -696,6 +805,97 @@ mod tests { ); } + async fn setup_table() -> DeltaTable { + use arrow_schema::{DataType, Field}; + let schema = Arc::new(ArrowSchema::new(vec![Field::new( + "id", + DataType::Utf8, + false, + )])); + + let data = + vec![Arc::new(arrow::array::StringArray::from(vec!["A", "B", "C", "D"])) as ArrayRef]; + let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; + + let table = DeltaOps::new_in_memory() + .write(batches.clone()) + .await + .unwrap(); + + DeltaOps(table) + .write(batches) + .with_save_mode(crate::protocol::SaveMode::Overwrite) + .await + .unwrap() + } + + #[tokio::test] + async fn test_cleanup_no_checkpoints() { + // Test that metadata clean up does not corrupt the table when no checkpoints exist + let table = setup_table().await; + + let log_retention_timestamp = (Utc::now().timestamp_millis() + + Duration::days(31).num_milliseconds()) + - table + .snapshot() + .unwrap() + .table_config() + .log_retention_duration() + .as_millis() as i64; + let count = cleanup_expired_logs_for( + table.version(), + table.log_store().as_ref(), + log_retention_timestamp, + ) + .await + .unwrap(); + assert_eq!(count, 0); + println!("{:?}", count); + + let path = Path::from("_delta_log/00000000000000000000.json"); + let res = table.log_store().object_store().get(&path).await; + assert!(res.is_ok()); + } + + #[tokio::test] + async fn test_cleanup_with_checkpoints() { + let table = setup_table().await; + create_checkpoint(&table).await.unwrap(); + + let log_retention_timestamp = (Utc::now().timestamp_millis() + + Duration::days(32).num_milliseconds()) + - table + .snapshot() + .unwrap() + .table_config() + .log_retention_duration() + .as_millis() as i64; + let count = cleanup_expired_logs_for( + table.version(), + table.log_store().as_ref(), + log_retention_timestamp, + ) + .await + .unwrap(); + assert_eq!(count, 1); + + let log_store = table.log_store(); + + let path = log_store.log_path().child("00000000000000000000.json"); + let res = table.log_store().object_store().get(&path).await; + assert!(res.is_err()); + + let path = log_store + .log_path() + .child("00000000000000000001.checkpoint.parquet"); + let res = table.log_store().object_store().get(&path).await; + assert!(res.is_ok()); + + let path = log_store.log_path().child("00000000000000000001.json"); + let res = table.log_store().object_store().get(&path).await; + assert!(res.is_ok()); + } + #[test] fn apply_stats_conversion_test() { let mut stats = STATS_JSON.clone(); @@ -785,6 +985,43 @@ mod tests { ); } + #[tokio::test] + async fn test_struct_with_single_list_field() { + // you need another column otherwise the entire stats struct is empty + // which also fails parquet write during checkpoint + let other_column_array: ArrayRef = Arc::new(Int32Array::from(vec![1])); + + let mut list_item_builder = Int32Builder::new(); + list_item_builder.append_value(1); + + let mut list_in_struct_builder = ListBuilder::new(list_item_builder); + list_in_struct_builder.append(true); + + let mut struct_builder = StructBuilder::new( + vec![arrow_schema::Field::new( + "list_in_struct", + arrow_schema::DataType::List(Arc::new(arrow_schema::Field::new( + "item", + arrow_schema::DataType::Int32, + true, + ))), + true, + )], + vec![Box::new(list_in_struct_builder)], + ); + struct_builder.append(true); + + let struct_with_list_array: ArrayRef = Arc::new(struct_builder.finish()); + let batch = RecordBatch::try_from_iter(vec![ + ("other_column", other_column_array), + ("struct_with_list", struct_with_list_array), + ]) + .unwrap(); + let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + + create_checkpoint(&table).await.unwrap(); + } + lazy_static! { static ref SCHEMA: Value = json!({ "type": "struct", diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/core/src/protocol/mod.rs similarity index 93% rename from crates/deltalake-core/src/protocol/mod.rs rename to crates/core/src/protocol/mod.rs index 661d75b244..7267529b74 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/core/src/protocol/mod.rs @@ -2,19 +2,13 @@ #![allow(non_camel_case_types)] -#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod checkpoints; -#[cfg(feature = "parquet2")] -pub mod parquet2_read; -#[cfg(feature = "parquet")] mod parquet_read; mod time_utils; -#[cfg(feature = "arrow")] use arrow_schema::ArrowError; use futures::StreamExt; use lazy_static::lazy_static; -use log::debug; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use regex::Regex; use serde::{Deserialize, Serialize}; @@ -24,12 +18,12 @@ use std::collections::HashMap; use std::hash::{Hash, Hasher}; use std::mem::take; use std::str::FromStr; +use tracing::{debug, error}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Add, CommitInfo, Metadata, Protocol, Remove}; use crate::logstore::LogStore; use crate::table::CheckPoint; -use crate::table::DeltaTableMetaData; /// Error returned when an invalid Delta log action is encountered. #[allow(missing_docs)] @@ -60,16 +54,10 @@ pub enum ProtocolError { #[error("Generic action error: {0}")] Generic(String), - #[cfg(any(feature = "parquet", feature = "parquet2"))] /// Error returned when parsing checkpoint parquet using the parquet crate. #[error("Failed to parse parquet checkpoint: {source}")] ParquetParseError { /// Parquet error details returned when parsing the checkpoint parquet - #[cfg(feature = "parquet2")] - #[from] - source: parquet2::error::Error, - /// Parquet error details returned when parsing the checkpoint parquet - #[cfg(feature = "parquet")] #[from] source: parquet::errors::ParquetError, }, @@ -83,7 +71,6 @@ pub enum ProtocolError { }, /// Error returned when converting the schema to Arrow format failed. - #[cfg(feature = "arrow")] #[error("Failed to convert into Arrow schema: {}", .source)] Arrow { /// Arrow error details returned when converting the schema in Arrow format failed @@ -233,18 +220,10 @@ pub struct StatsParsed { // start of per column stats /// Contains a value smaller than all values present in the file for all columns. - #[cfg(feature = "parquet")] pub min_values: HashMap, - /// Contains a value smaller than all values present in the file for all columns. - #[cfg(feature = "parquet2")] - pub min_values: HashMap, /// Contains a value larger than all values present in the file for all columns. - #[cfg(feature = "parquet")] /// Contains a value larger than all values present in the file for all columns. pub max_values: HashMap, - #[cfg(feature = "parquet2")] - /// Contains a value larger than all values present in the file for all columns. - pub max_values: HashMap, /// The number of null values for all columns. pub null_count: HashMap, } @@ -272,13 +251,12 @@ impl Eq for Add {} impl Add { /// Get whatever stats are available. Uses (parquet struct) parsed_stats if present falling back to json stats. - #[cfg(any(feature = "parquet", feature = "parquet2"))] - pub fn get_stats(&self) -> Result, serde_json::error::Error> { + pub(crate) fn get_stats(&self) -> Result, serde_json::error::Error> { match self.get_stats_parsed() { Ok(Some(stats)) => Ok(Some(stats)), Ok(None) => self.get_json_stats(), Err(e) => { - log::error!( + error!( "Error when reading parquet stats {:?} {e}. Attempting to read json stats", self.stats_parsed ); @@ -287,15 +265,9 @@ impl Add { } } - /// Get whatever stats are available. - #[cfg(not(any(feature = "parquet", feature = "parquet2")))] - pub fn get_stats(&self) -> Result, serde_json::error::Error> { - self.get_json_stats() - } - /// Returns the serde_json representation of stats contained in the action if present. /// Since stats are defined as optional in the protocol, this may be None. - pub fn get_json_stats(&self) -> Result, serde_json::error::Error> { + fn get_json_stats(&self) -> Result, serde_json::error::Error> { let ps: Result, serde_json::error::Error> = self .stats .as_ref() @@ -336,25 +308,6 @@ impl PartialEq for Remove { } } -impl TryFrom for Metadata { - type Error = ProtocolError; - - fn try_from(metadata: DeltaTableMetaData) -> Result { - let schema_string = serde_json::to_string(&metadata.schema) - .map_err(|source| ProtocolError::SerializeOperation { source })?; - Ok(Self { - id: metadata.id, - name: metadata.name, - description: metadata.description, - format: metadata.format, - schema_string, - partition_columns: metadata.partition_columns, - created_time: metadata.created_time, - configuration: metadata.configuration, - }) - } -} - #[allow(clippy::large_enum_variant)] #[derive(Serialize, Deserialize, Debug, Clone)] #[serde(rename_all = "camelCase")] @@ -384,7 +337,7 @@ pub enum DeltaOperation { /// The min reader and writer protocol versions of the table protocol: Protocol, /// Metadata associated with the new table - metadata: DeltaTableMetaData, + metadata: Metadata, }, /// Represents a Delta `Write` operation. @@ -410,6 +363,19 @@ pub enum DeltaOperation { /// The update predicate predicate: Option, }, + /// Add constraints to a table + AddConstraint { + /// Constraints name + name: String, + /// Expression to check against + expr: String, + }, + + /// Drops constraints from a table + DropConstraint { + /// Constraints name + name: String, + }, /// Merge data with a source data with the following predicate #[serde(rename_all = "camelCase")] @@ -497,6 +463,8 @@ impl DeltaOperation { DeltaOperation::Restore { .. } => "RESTORE", DeltaOperation::VacuumStart { .. } => "VACUUM START", DeltaOperation::VacuumEnd { .. } => "VACUUM END", + DeltaOperation::AddConstraint { .. } => "ADD CONSTRAINT", + DeltaOperation::DropConstraint { .. } => "DROP CONSTRAINT", } } @@ -532,7 +500,11 @@ 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 { .. } + | Self::DropConstraint { .. } => false, Self::Create { .. } | Self::FileSystemCheck {} | Self::StreamingUpdate { .. } @@ -648,7 +620,7 @@ pub(crate) async fn find_latest_check_point_for_version( let mut cp: Option = None; let object_store = log_store.object_store(); - let mut stream = object_store.list(Some(log_store.log_path())).await?; + let mut stream = object_store.list(Some(log_store.log_path())); while let Some(obj_meta) = stream.next().await { // Exit early if any objects can't be listed. @@ -713,7 +685,6 @@ mod tests { data_change: true, deletion_vector: None, partition_values: Default::default(), - partition_values_parsed: None, stats_parsed: None, tags: None, size: 0, @@ -789,7 +760,6 @@ mod tests { data_change: true, deletion_vector: None, partition_values: Default::default(), - partition_values_parsed: None, stats_parsed: None, tags: None, size: 0, @@ -873,7 +843,6 @@ mod tests { serde_json::from_str(buf).expect("Expected to be able to deserialize"); } - #[cfg(feature = "arrow")] mod arrow_tests { use arrow::array::{self, ArrayRef, StructArray}; use arrow::compute::kernels::cast_utils::Parser; @@ -901,9 +870,9 @@ mod tests { #[tokio::test] async fn test_with_partitions() { // test table with partitions - let path = "./tests/data/delta-0.8.0-null-partition"; + let path = "../test/tests/data/delta-0.8.0-null-partition"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let mut expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -922,7 +891,7 @@ mod tests { assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(false).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); expected_columns[4] = ( @@ -940,9 +909,9 @@ mod tests { #[tokio::test] async fn test_with_deletion_vector() { // test table with partitions - let path = "./tests/data/table_with_deletion_logs"; + let path = "../test/tests/data/table_with_deletion_logs"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let actions = actions .project(&[ @@ -1000,7 +969,7 @@ mod tests { assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(false).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let actions = actions .project(&[ @@ -1046,10 +1015,10 @@ mod tests { #[tokio::test] async fn test_without_partitions() { // test table without partitions - let path = "./tests/data/simple_table"; + let path = "../test/tests/data/simple_table"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -1088,7 +1057,7 @@ mod tests { assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(false).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); // For now, this column is ignored. @@ -1104,9 +1073,9 @@ mod tests { #[tokio::test] async fn test_with_column_mapping() { // test table with column mapping and partitions - let path = "./tests/data/table_with_column_mapping"; + let path = "../test/tests/data/table_with_column_mapping"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(true).unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ ( "path", @@ -1178,9 +1147,9 @@ mod tests { #[tokio::test] async fn test_with_stats() { // test table with stats - let path = "./tests/data/delta-0.8.0"; + let path = "../test/tests/data/delta-0.8.0"; let table = crate::open_table(path).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(true).unwrap(); let actions = sort_batch_by(&actions, "path").unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ @@ -1222,11 +1191,11 @@ mod tests { #[tokio::test] async fn test_only_struct_stats() { // test table with no json stats - let path = "./tests/data/delta-1.2.1-only-struct-stats"; + let path = "../test/tests/data/delta-1.2.1-only-struct-stats"; let mut table = crate::open_table(path).await.unwrap(); table.load_version(1).await.unwrap(); - let actions = table.get_state().add_actions_table(true).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(true).unwrap(); let expected_columns: Vec<(&str, ArrayRef)> = vec![ ( @@ -1337,15 +1306,21 @@ mod tests { ), ( "min.timestamp", - Arc::new(array::TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2022-10-24T22:59:32.846Z"), - ])), + Arc::new( + array::TimestampMicrosecondArray::from(vec![ + TimestampMicrosecondType::parse("2022-10-24T22:59:32.846Z"), + ]) + .with_timezone("UTC"), + ), ), ( "max.timestamp", - Arc::new(array::TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2022-10-24T22:59:32.846Z"), - ])), + Arc::new( + array::TimestampMicrosecondArray::from(vec![ + TimestampMicrosecondType::parse("2022-10-24T22:59:32.846Z"), + ]) + .with_timezone("UTC"), + ), ), ( "null_count.struct.struct_element", @@ -1407,7 +1382,7 @@ mod tests { ); assert_eq!(expected, actions); - let actions = table.get_state().add_actions_table(false).unwrap(); + let actions = table.snapshot().unwrap().add_actions_table(false).unwrap(); // For brevity, just checking a few nested columns in stats assert_eq!( diff --git a/crates/deltalake-core/src/protocol/parquet_read/mod.rs b/crates/core/src/protocol/parquet_read/mod.rs similarity index 91% rename from crates/deltalake-core/src/protocol/parquet_read/mod.rs rename to crates/core/src/protocol/parquet_read/mod.rs index 21ad2bdff8..90d43b147c 100644 --- a/crates/deltalake-core/src/protocol/parquet_read/mod.rs +++ b/crates/core/src/protocol/parquet_read/mod.rs @@ -5,7 +5,9 @@ use num_bigint::BigInt; use num_traits::cast::ToPrimitive; use parquet::record::{Field, ListAccessor, MapAccessor, RowAccessor}; use serde_json::json; +use tracing::{debug, error, warn}; +use crate::kernel::models::actions::serde_path::decode_path; use crate::kernel::{ Action, Add, AddCDCFile, DeletionVectorDescriptor, Metadata, Protocol, Remove, StorageType, Txn, }; @@ -86,10 +88,9 @@ impl DeletionVectorDescriptor { })?; } _ => { - log::debug!( + debug!( "Unexpected field name `{}` for deletion vector: {:?}", - name, - record + name, record ); } } @@ -105,7 +106,6 @@ impl Add { size: -1, modification_time: -1, data_change: true, - partition_values_parsed: None, partition_values: HashMap::new(), stats: None, stats_parsed: None, @@ -119,10 +119,13 @@ impl Add { for (i, (name, _)) in record.get_column_iter().enumerate() { match name.as_str() { "path" => { - re.path = record - .get_string(i) - .map_err(|_| gen_action_type_error("add", "path", "string"))? - .clone(); + re.path = decode_path( + record + .get_string(i) + .map_err(|_| gen_action_type_error("add", "path", "string"))? + .clone() + .as_str(), + )?; } "size" => { re.size = record @@ -153,16 +156,16 @@ impl Add { )) })?; } - "partitionValues_parsed" => { - re.partition_values_parsed = Some( - record - .get_group(i) - .map_err(|_| { - gen_action_type_error("add", "partitionValues_parsed", "struct") - })? - .clone(), - ); - } + // "partitionValues_parsed" => { + // re.partition_values_parsed = Some( + // record + // .get_group(i) + // .map_err(|_| { + // gen_action_type_error("add", "partitionValues_parsed", "struct") + // })? + // .clone(), + // ); + // } "tags" => match record.get_map(i) { Ok(tags_map) => { let mut tags = HashMap::new(); @@ -204,10 +207,9 @@ impl Add { } }, _ => { - log::debug!( + debug!( "Unexpected field name `{}` for add action: {:?}", - name, - record + name, record ); } } @@ -227,7 +229,7 @@ impl Add { "numRecords" => if let Ok(v) = record.get_long(i) { stats.num_records = v; } else { - log::error!("Expect type of stats_parsed field numRecords to be long, got: {}", record); + error!("Expect type of stats_parsed field numRecords to be long, got: {}", record); } "minValues" => if let Ok(row) = record.get_group(i) { for (name, field) in row.get_column_iter() { @@ -238,7 +240,7 @@ impl Add { } } } else { - log::error!("Expect type of stats_parsed field minRecords to be struct, got: {}", record); + error!("Expect type of stats_parsed field minRecords to be struct, got: {}", record); } "maxValues" => if let Ok(row) = record.get_group(i) { for (name, field) in row.get_column_iter() { @@ -249,7 +251,7 @@ impl Add { } } } else { - log::error!("Expect type of stats_parsed field maxRecords to be struct, got: {}", record); + error!("Expect type of stats_parsed field maxRecords to be struct, got: {}", record); } "nullCount" => if let Ok(row) = record.get_group(i) { for (name, field) in row.get_column_iter() { @@ -260,10 +262,10 @@ impl Add { } } } else { - log::error!("Expect type of stats_parsed field nullCount to be struct, got: {}", record); + error!("Expect type of stats_parsed field nullCount to be struct, got: {}", record); } _ => { - log::debug!( + debug!( "Unexpected field name `{}` for stats_parsed: {:?}", name, record, @@ -289,10 +291,9 @@ fn field_to_value_stat(field: &Field, field_name: &str) -> Option Option Some(ColumnCountStat::Value(*value)), _ => { - log::warn!( + warn!( "Unexpected type when parsing nullCounts for {}. Found {}", - field_name, - field + field_name, field ); None } @@ -433,12 +433,10 @@ impl Metadata { .map_err(|_| gen_action_type_error("metaData", "schemaString", "string"))? .clone(); } - "createdTime" => { - re.created_time = - Some(record.get_long(i).map_err(|_| { - gen_action_type_error("metaData", "createdTime", "long") - })?); - } + "createdTime" => match record.get_long(i) { + Ok(s) => re.created_time = Some(s), + _ => re.created_time = None, + }, "configuration" => { let configuration_map = record .get_map(i) @@ -484,10 +482,9 @@ impl Metadata { } } _ => { - log::debug!( + debug!( "Unexpected field name `{}` for metaData action: {:?}", - name, - record + name, record ); } } @@ -515,10 +512,13 @@ impl Remove { for (i, (name, _)) in record.get_column_iter().enumerate() { match name.as_str() { "path" => { - re.path = record - .get_string(i) - .map_err(|_| gen_action_type_error("remove", "path", "string"))? - .clone(); + re.path = decode_path( + record + .get_string(i) + .map_err(|_| gen_action_type_error("remove", "path", "string"))? + .clone() + .as_str(), + )?; } "dataChange" => { re.data_change = record @@ -572,10 +572,9 @@ impl Remove { } "numRecords" => {} _ => { - log::debug!( + debug!( "Unexpected field name `{}` for remove action: {:?}", - name, - record + name, record ); } } @@ -608,10 +607,9 @@ impl Txn { re.last_updated = record.get_long(i).map(Some).unwrap_or(None); } _ => { - log::debug!( + debug!( "Unexpected field name `{}` for txn action: {:?}", - name, - record + name, record ); } } @@ -655,10 +653,9 @@ impl Protocol { .ok() } _ => { - log::debug!( + debug!( "Unexpected field name `{}` for protocol action: {:?}", - name, - record + name, record ); } } @@ -729,7 +726,8 @@ mod tests { use parquet::file::reader::{FileReader, SerializedFileReader}; use std::fs::File; - let path = "./tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet"; + let path = + "../test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet"; let preader = SerializedFileReader::new(File::open(path).unwrap()).unwrap(); let mut iter = preader.get_row_iter(None).unwrap(); diff --git a/crates/core/src/protocol/time_utils.rs b/crates/core/src/protocol/time_utils.rs new file mode 100644 index 0000000000..72465ea7a5 --- /dev/null +++ b/crates/core/src/protocol/time_utils.rs @@ -0,0 +1,54 @@ +//! Utility functions for converting time formats. +#![allow(unused)] + +use arrow::temporal_conversions; +use parquet::basic::TimeUnit; + +/// Convert an ISO-8601/RFC3339 timestamp string to a numeric microsecond epoch representation. +/// Stats strings are written with millisecond precision as described by the delta protocol. +pub fn timestamp_micros_from_stats_string(s: &str) -> Result { + chrono::DateTime::parse_from_rfc3339(s).map(|dt| dt.timestamp_millis() * 1000) +} + +/// Convert the timestamp to a ISO-8601 style format suitable for JSON statistics. +pub fn timestamp_to_delta_stats_string(n: i64, time_unit: &TimeUnit) -> Option { + let dt = match time_unit { + TimeUnit::MILLIS(_) => temporal_conversions::timestamp_ms_to_datetime(n), + TimeUnit::MICROS(_) => temporal_conversions::timestamp_us_to_datetime(n), + TimeUnit::NANOS(_) => temporal_conversions::timestamp_ns_to_datetime(n), + }?; + + Some(format!("{}", dt.format("%Y-%m-%dT%H:%M:%S%.3fZ"))) +} + +#[cfg(test)] +mod tests { + use super::*; + use parquet::format::{MicroSeconds, MilliSeconds, NanoSeconds, TimeUnit}; + + #[test] + fn test_timestamp_to_delta_stats_string() { + let s = + timestamp_to_delta_stats_string(1628685199541, &TimeUnit::MILLIS(MilliSeconds::new())) + .unwrap(); + assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); + let s = timestamp_to_delta_stats_string( + 1628685199541000, + &TimeUnit::MICROS(MicroSeconds::new()), + ) + .unwrap(); + assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); + let s = timestamp_to_delta_stats_string( + 1628685199541000000, + &TimeUnit::NANOS(NanoSeconds::new()), + ) + .unwrap(); + assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); + } + + #[test] + fn test_timestamp_micros_from_stats_string() { + let us = timestamp_micros_from_stats_string("2021-08-11T12:33:19.541Z").unwrap(); + assert_eq!(1628685199541000i64, us); + } +} diff --git a/crates/deltalake-core/src/schema/mod.rs b/crates/core/src/schema/mod.rs similarity index 100% rename from crates/deltalake-core/src/schema/mod.rs rename to crates/core/src/schema/mod.rs diff --git a/crates/deltalake-core/src/schema/partitions.rs b/crates/core/src/schema/partitions.rs similarity index 65% rename from crates/deltalake-core/src/schema/partitions.rs rename to crates/core/src/schema/partitions.rs index 4e8830596c..3ecd5cbdad 100644 --- a/crates/deltalake-core/src/schema/partitions.rs +++ b/crates/core/src/schema/partitions.rs @@ -1,13 +1,12 @@ //! Delta Table partition handling logic. - +//! +use serde::{Serialize, Serializer}; +use std::cmp::Ordering; +use std::collections::HashMap; use std::convert::TryFrom; -use chrono::{NaiveDateTime, ParseResult}; - use crate::errors::DeltaTableError; -use crate::kernel::{DataType, PrimitiveType}; -use std::cmp::Ordering; -use std::collections::HashMap; +use crate::kernel::{DataType, PrimitiveType, Scalar}; /// A special value used in Hive to represent the null partition in partitioned tables pub const NULL_PARTITION_VALUE_DATA_PATH: &str = "__HIVE_DEFAULT_PARTITION__"; @@ -42,47 +41,18 @@ pub struct PartitionFilter { pub value: PartitionValue, } -fn parse_timestamp(timestamp_str: &str) -> ParseResult { - // Timestamp format as per https://github.com/delta-io/delta/blob/master/PROTOCOL.md#partition-value-serialization - let format = "%Y-%m-%d %H:%M:%S%.f"; - - NaiveDateTime::parse_from_str(timestamp_str, format) -} - fn compare_typed_value( - partition_value: &str, + partition_value: &Scalar, filter_value: &str, data_type: &DataType, ) -> Option { match data_type { - DataType::Primitive(primitive_type) => match primitive_type { - PrimitiveType::Long - | PrimitiveType::Integer - | PrimitiveType::Short - | PrimitiveType::Byte => match filter_value.parse::() { - Ok(parsed_filter_value) => { - let parsed_partition_value = partition_value.parse::().unwrap(); - parsed_partition_value.partial_cmp(&parsed_filter_value) - } - _ => None, - }, - PrimitiveType::Float | PrimitiveType::Double => match filter_value.parse::() { - Ok(parsed_filter_value) => { - let parsed_partition_value = partition_value.parse::().unwrap(); - parsed_partition_value.partial_cmp(&parsed_filter_value) - } - _ => None, - }, - PrimitiveType::Timestamp => match parse_timestamp(filter_value) { - Ok(parsed_filter_value) => { - let parsed_partition_value = parse_timestamp(partition_value).unwrap(); - parsed_partition_value.partial_cmp(&parsed_filter_value) - } - _ => None, - }, - _ => partition_value.partial_cmp(filter_value), - }, - _ => partition_value.partial_cmp(filter_value), + DataType::Primitive(primitive_type) => { + let other = primitive_type.parse_scalar(filter_value).ok()?; + partition_value.partial_cmp(&other) + } + // NOTE: complex types are not supported as partition columns + _ => None, } } @@ -93,6 +63,9 @@ impl PartitionFilter { if self.key != partition.key { return false; } + if self.value == PartitionValue::Equal("".to_string()) { + return partition.value.is_null(); + } match &self.value { PartitionValue::Equal(value) => { @@ -101,7 +74,7 @@ impl PartitionFilter { .map(|x| x.is_eq()) .unwrap_or(false) } else { - value == &partition.value + partition.value.serialize() == *value } } PartitionValue::NotEqual(value) => { @@ -110,7 +83,7 @@ impl PartitionFilter { .map(|x| !x.is_eq()) .unwrap_or(false) } else { - value != &partition.value + !(partition.value.serialize() == *value) } } PartitionValue::GreaterThan(value) => { @@ -133,8 +106,8 @@ impl PartitionFilter { .map(|x| x.is_le()) .unwrap_or(false) } - PartitionValue::In(value) => value.contains(&partition.value), - PartitionValue::NotIn(value) => !value.contains(&partition.value), + PartitionValue::In(value) => value.contains(&partition.value.serialize()), + PartitionValue::NotIn(value) => !value.contains(&partition.value.serialize()), } } @@ -152,6 +125,36 @@ impl PartitionFilter { } } +/// Create desired string representation for PartitionFilter. +/// Used in places like predicate in operationParameters, etc. +impl Serialize for PartitionFilter { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + let s = match &self.value { + PartitionValue::Equal(value) => format!("{} = '{}'", self.key, value), + PartitionValue::NotEqual(value) => format!("{} != '{}'", self.key, value), + PartitionValue::GreaterThan(value) => format!("{} > '{}'", self.key, value), + PartitionValue::GreaterThanOrEqual(value) => format!("{} >= '{}'", self.key, value), + PartitionValue::LessThan(value) => format!("{} < '{}'", self.key, value), + PartitionValue::LessThanOrEqual(value) => format!("{} <= '{}'", self.key, value), + // used upper case for IN and NOT similar to SQL + PartitionValue::In(values) => { + let quoted_values: Vec = + values.iter().map(|v| format!("'{}'", v)).collect(); + format!("{} IN ({})", self.key, quoted_values.join(", ")) + } + PartitionValue::NotIn(values) => { + let quoted_values: Vec = + values.iter().map(|v| format!("'{}'", v)).collect(); + format!("{} NOT IN ({})", self.key, quoted_values.join(", ")) + } + }; + serializer.serialize_str(&s) + } +} + /// Create a PartitionFilter from a filter Tuple with the structure (key, operation, value). impl TryFrom<(&str, &str, &str)> for PartitionFilter { type Error = DeltaTableError; @@ -215,67 +218,20 @@ impl TryFrom<(&str, &str, &[&str])> for PartitionFilter { } /// A Struct DeltaTablePartition used to represent a partition of a DeltaTable. -#[derive(Clone, Debug, PartialEq, Eq)] +#[derive(Clone, Debug, PartialEq)] pub struct DeltaTablePartition { /// The key of the DeltaTable partition. pub key: String, /// The value of the DeltaTable partition. - pub value: String, + pub value: Scalar, } -/// Create a DeltaTable partition from a HivePartition string. -/// -/// A HivePartition string is represented by a "key=value" format. -/// -/// ```rust -/// use deltalake_core::DeltaTablePartition; -/// -/// let hive_part = "ds=2023-01-01"; -/// let partition = DeltaTablePartition::try_from(hive_part).unwrap(); -/// assert_eq!("ds", partition.key); -/// assert_eq!("2023-01-01", partition.value); -/// ``` -impl TryFrom<&str> for DeltaTablePartition { - type Error = DeltaTableError; - - /// Try to create a DeltaTable partition from a HivePartition string. - /// Returns a DeltaTableError if the string is not in the form of a HivePartition. - fn try_from(partition: &str) -> Result { - let partition_splitted: Vec<&str> = partition.split('=').collect(); - match partition_splitted { - partition_splitted if partition_splitted.len() == 2 => Ok(DeltaTablePartition { - key: partition_splitted[0].to_owned(), - value: partition_splitted[1].to_owned(), - }), - _ => Err(DeltaTableError::PartitionError { - partition: partition.to_string(), - }), - } - } -} +impl Eq for DeltaTablePartition {} impl DeltaTablePartition { - /// Try to create a DeltaTable partition from a partition value kv pair. - /// - /// ```rust - /// use deltalake_core::DeltaTablePartition; - /// - /// let value = ("ds", &Some("2023-01-01".to_string())); - /// let null_default = "1979-01-01"; - /// let partition = DeltaTablePartition::from_partition_value(value, null_default); - /// - /// assert_eq!("ds", partition.key); - /// assert_eq!("2023-01-01", partition.value); - /// ``` - pub fn from_partition_value( - partition_value: (&str, &Option), - default_for_null: &str, - ) -> Self { + /// Create a DeltaTable partition from a Tuple of (key, value). + pub fn from_partition_value(partition_value: (&str, &Scalar)) -> Self { let (k, v) = partition_value; - let v = match v { - Some(s) => s, - None => default_for_null, - }; DeltaTablePartition { key: k.to_owned(), value: v.to_owned(), @@ -286,11 +242,51 @@ impl DeltaTablePartition { #[cfg(test)] mod tests { use super::*; + use serde_json::json; + + fn check_json_serialize(filter: PartitionFilter, expected_json: &str) { + assert_eq!(serde_json::to_value(&filter).unwrap(), json!(expected_json)) + } #[test] - fn tryfrom_invalid() { - let buf = "this-is-not-a-partition"; - let partition = DeltaTablePartition::try_from(buf); - assert!(partition.is_err()); + fn test_serialize_partition_filter() { + check_json_serialize( + PartitionFilter::try_from(("date", "=", "2022-05-22")).unwrap(), + "date = '2022-05-22'", + ); + check_json_serialize( + PartitionFilter::try_from(("date", "!=", "2022-05-22")).unwrap(), + "date != '2022-05-22'", + ); + check_json_serialize( + PartitionFilter::try_from(("date", ">", "2022-05-22")).unwrap(), + "date > '2022-05-22'", + ); + check_json_serialize( + PartitionFilter::try_from(("date", ">=", "2022-05-22")).unwrap(), + "date >= '2022-05-22'", + ); + check_json_serialize( + PartitionFilter::try_from(("date", "<", "2022-05-22")).unwrap(), + "date < '2022-05-22'", + ); + check_json_serialize( + PartitionFilter::try_from(("date", "<=", "2022-05-22")).unwrap(), + "date <= '2022-05-22'", + ); + check_json_serialize( + PartitionFilter::try_from(("date", "in", vec!["2023-11-04", "2023-06-07"].as_slice())) + .unwrap(), + "date IN ('2023-11-04', '2023-06-07')", + ); + check_json_serialize( + PartitionFilter::try_from(( + "date", + "not in", + vec!["2023-11-04", "2023-06-07"].as_slice(), + )) + .unwrap(), + "date NOT IN ('2023-11-04', '2023-06-07')", + ); } } diff --git a/crates/deltalake-core/src/storage/file.rs b/crates/core/src/storage/file.rs similarity index 94% rename from crates/deltalake-core/src/storage/file.rs rename to crates/core/src/storage/file.rs index 6e64e52be9..c63a00dae6 100644 --- a/crates/deltalake-core/src/storage/file.rs +++ b/crates/core/src/storage/file.rs @@ -6,7 +6,7 @@ use bytes::Bytes; use futures::stream::BoxStream; use object_store::{ local::LocalFileSystem, path::Path as ObjectStorePath, Error as ObjectStoreError, GetOptions, - GetResult, ListResult, MultipartId, ObjectMeta as ObjStoreObjectMeta, ObjectStore, + GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, PutOptions, PutResult, Result as ObjectStoreResult, }; use std::ops::Range; @@ -166,10 +166,19 @@ impl std::fmt::Display for FileStorageBackend { #[async_trait::async_trait] impl ObjectStore for FileStorageBackend { - async fn put(&self, location: &ObjectStorePath, bytes: Bytes) -> ObjectStoreResult<()> { + async fn put(&self, location: &ObjectStorePath, bytes: Bytes) -> ObjectStoreResult { self.inner.put(location, bytes).await } + async fn put_opts( + &self, + location: &ObjectStorePath, + bytes: Bytes, + options: PutOptions, + ) -> ObjectStoreResult { + self.inner.put_opts(location, bytes, options).await + } + async fn get(&self, location: &ObjectStorePath) -> ObjectStoreResult { self.inner.get(location).await } @@ -190,7 +199,7 @@ impl ObjectStore for FileStorageBackend { self.inner.get_range(location, range).await } - async fn head(&self, location: &ObjectStorePath) -> ObjectStoreResult { + async fn head(&self, location: &ObjectStorePath) -> ObjectStoreResult { self.inner.head(location).await } @@ -198,11 +207,19 @@ impl ObjectStore for FileStorageBackend { self.inner.delete(location).await } - async fn list( + fn list( + &self, + prefix: Option<&ObjectStorePath>, + ) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list(prefix) + } + + fn list_with_offset( &self, prefix: Option<&ObjectStorePath>, - ) -> ObjectStoreResult>> { - self.inner.list(prefix).await + offset: &ObjectStorePath, + ) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list_with_offset(prefix, offset) } async fn list_with_delimiter( @@ -379,9 +396,9 @@ mod tests { use std::io::Write; use std::path::{Path, PathBuf}; - #[tokio::test()] + #[tokio::test] async fn test_rename_noreplace() { - let tmp_dir = tempdir::TempDir::new_in(".", "test_rename_noreplace").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let a = create_file(tmp_dir.path(), "a"); let b = create_file(tmp_dir.path(), "b"); let c = &tmp_dir.path().join("c"); diff --git a/crates/core/src/storage/mod.rs b/crates/core/src/storage/mod.rs new file mode 100644 index 0000000000..fc6de9a773 --- /dev/null +++ b/crates/core/src/storage/mod.rs @@ -0,0 +1,174 @@ +//! Object storage backend abstraction layer for Delta Table transaction logs and data + +use dashmap::DashMap; +use std::collections::HashMap; +use std::sync::{Arc, OnceLock}; + +use lazy_static::lazy_static; +use serde::{Deserialize, Serialize}; +use url::Url; + +pub mod file; +pub mod retry_ext; +pub mod utils; + +use crate::{DeltaResult, DeltaTableError}; + +pub use object_store; +use object_store::local::LocalFileSystem; +use object_store::memory::InMemory; +pub use object_store::path::{Path, DELIMITER}; +use object_store::prefix::PrefixStore; +pub use object_store::{ + DynObjectStore, Error as ObjectStoreError, GetResult, ListResult, MultipartId, ObjectMeta, + ObjectStore, Result as ObjectStoreResult, +}; +pub use retry_ext::ObjectStoreRetryExt; +pub use utils::*; + +lazy_static! { + static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); +} + +/// Sharable reference to [`ObjectStore`] +pub type ObjectStoreRef = Arc; + +/// Factory trait for creating [ObjectStoreRef] instances at runtime +pub trait ObjectStoreFactory: Send + Sync { + #[allow(missing_docs)] + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)>; +} + +#[derive(Clone, Debug, Default)] +pub(crate) struct DefaultObjectStoreFactory {} + +impl ObjectStoreFactory for DefaultObjectStoreFactory { + fn parse_url_opts( + &self, + url: &Url, + _options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + match url.scheme() { + "memory" => { + let path = Path::from_url_path(url.path())?; + let store: Arc = Arc::new(InMemory::new()) as ObjectStoreRef; + Ok((url_prefix_handler(store, path.clone())?, path)) + } + "file" => { + let store = Arc::new(LocalFileSystem::new_with_prefix( + url.to_file_path().unwrap(), + )?) as ObjectStoreRef; + Ok((store, Path::from("/"))) + } + _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), + } + } +} + +/// TODO +pub type FactoryRegistry = Arc>>; + +/// TODO +pub fn factories() -> FactoryRegistry { + static REGISTRY: OnceLock = OnceLock::new(); + REGISTRY + .get_or_init(|| { + let registry = FactoryRegistry::default(); + registry.insert( + Url::parse("memory://").unwrap(), + Arc::new(DefaultObjectStoreFactory::default()), + ); + registry.insert( + Url::parse("file://").unwrap(), + Arc::new(DefaultObjectStoreFactory::default()), + ); + registry + }) + .clone() +} + +/// Simpler access pattern for the [FactoryRegistry] to get a single store +pub fn store_for(url: &Url) -> DeltaResult { + let scheme = Url::parse(&format!("{}://", url.scheme())).unwrap(); + if let Some(factory) = factories().get(&scheme) { + let (store, _prefix) = factory.parse_url_opts(url, &StorageOptions::default())?; + Ok(store) + } else { + Err(DeltaTableError::InvalidTableLocation(url.clone().into())) + } +} + +/// Options used for configuring backend storage +#[derive(Clone, Debug, Serialize, Deserialize, Default)] +pub struct StorageOptions(pub HashMap); + +impl From> for StorageOptions { + fn from(value: HashMap) -> Self { + Self(value) + } +} + +/// Return the uri of commit version. +/// +/// ```rust +/// # use deltalake_core::storage::*; +/// use object_store::path::Path; +/// let uri = commit_uri_from_version(1); +/// assert_eq!(uri, Path::from("_delta_log/00000000000000000001.json")); +/// ``` +pub fn commit_uri_from_version(version: i64) -> Path { + let version = format!("{version:020}.json"); + DELTA_LOG_PATH.child(version.as_str()) +} + +/// Return true for all the stringly values typically associated with true +/// +/// aka YAML booleans +/// +/// ```rust +/// # use deltalake_core::storage::*; +/// for value in ["1", "true", "on", "YES", "Y"] { +/// assert!(str_is_truthy(value)); +/// } +/// for value in ["0", "FALSE", "off", "NO", "n", "bork"] { +/// assert!(!str_is_truthy(value)); +/// } +/// ``` +pub fn str_is_truthy(val: &str) -> bool { + val.eq_ignore_ascii_case("1") + | val.eq_ignore_ascii_case("true") + | val.eq_ignore_ascii_case("on") + | val.eq_ignore_ascii_case("yes") + | val.eq_ignore_ascii_case("y") +} + +/// Simple function to wrap the given [ObjectStore] in a [PrefixStore] if necessary +/// +/// This simplifies the use of t he storage since it ensures that list/get/etc operations +/// start from the prefix in the object storage rather than from the root configured URI of the +/// [ObjectStore] +pub fn url_prefix_handler(store: T, prefix: Path) -> DeltaResult { + if prefix != Path::from("/") { + Ok(Arc::new(PrefixStore::new(store, prefix))) + } else { + Ok(Arc::new(store)) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_url_prefix_handler() { + let store = InMemory::new(); + let path = Path::parse("/databases/foo/bar").expect("Failed to parse path"); + + let prefixed = url_prefix_handler(store, path); + assert!(prefixed.is_ok()); + } +} diff --git a/crates/core/src/storage/retry_ext.rs b/crates/core/src/storage/retry_ext.rs new file mode 100644 index 0000000000..b9f7ebf691 --- /dev/null +++ b/crates/core/src/storage/retry_ext.rs @@ -0,0 +1,82 @@ +//! Retry extension for [`ObjectStore`] + +use bytes::Bytes; +use object_store::{path::Path, Error, ObjectStore, PutResult, Result}; +use tracing::log::*; + +/// Retry extension for [`ObjectStore`] +/// +/// Read-only operations are retried by [`ObjectStore`] internally. However, PUT/DELETE operations +/// are not retried even thought they are technically idempotent. [`ObjectStore`] does not retry +/// those operations because having preconditions may produce different results for the same +/// request. PUT/DELETE operations without preconditions are idempotent and can be retried. +/// Unfortunately, [`ObjectStore`]'s retry mechanism only works on HTTP request level, thus there +/// is no way to distinguish whether a request has preconditions or not. +/// +/// This trait provides additional methods for working with [`ObjectStore`] that automatically retry +/// unconditional operations when they fail. +/// +/// See also: +/// - https://github.com/apache/arrow-rs/pull/5278 +#[async_trait::async_trait] +pub trait ObjectStoreRetryExt: ObjectStore { + /// Save the provided bytes to the specified location + /// + /// The operation is guaranteed to be atomic, it will either successfully write the entirety of + /// bytes to location, or fail. No clients should be able to observe a partially written object + /// + /// Note that `put_with_opts` may have precondition semantics, and thus may not be retriable. + async fn put_with_retries( + &self, + location: &Path, + bytes: Bytes, + max_retries: usize, + ) -> Result { + let mut attempt_number = 1; + while attempt_number <= max_retries { + match self.put(location, bytes.clone()).await { + Ok(result) => return Ok(result), + Err(err) if attempt_number == max_retries => { + return Err(err); + } + Err(Error::Generic { store, source }) => { + debug!( + "put_with_retries attempt {} failed: {} {}", + attempt_number, store, source + ); + attempt_number += 1; + } + Err(err) => { + return Err(err); + } + } + } + unreachable!("loop yields Ok or Err in body when attempt_number = max_retries") + } + + /// Delete the object at the specified location + async fn delete_with_retries(&self, location: &Path, max_retries: usize) -> Result<()> { + let mut attempt_number = 1; + while attempt_number <= max_retries { + match self.delete(location).await { + Ok(()) | Err(Error::NotFound { .. }) => return Ok(()), + Err(err) if attempt_number == max_retries => { + return Err(err); + } + Err(Error::Generic { store, source }) => { + debug!( + "delete_with_retries attempt {} failed: {} {}", + attempt_number, store, source + ); + attempt_number += 1; + } + Err(err) => { + return Err(err); + } + } + } + unreachable!("loop yields Ok or Err in body when attempt_number = max_retries") + } +} + +impl ObjectStoreRetryExt for T {} diff --git a/crates/deltalake-core/src/storage/utils.rs b/crates/core/src/storage/utils.rs similarity index 58% rename from crates/deltalake-core/src/storage/utils.rs rename to crates/core/src/storage/utils.rs index 7b8e76c47d..e4dde08387 100644 --- a/crates/deltalake-core/src/storage/utils.rs +++ b/crates/core/src/storage/utils.rs @@ -1,51 +1,12 @@ //! Utility functions for working across Delta tables -use std::collections::HashMap; -use std::sync::Arc; - use chrono::{NaiveDateTime, TimeZone, Utc}; -use futures::{StreamExt, TryStreamExt}; +use futures::TryStreamExt; use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::Add; -use crate::table::builder::DeltaTableBuilder; - -/// Copies the contents from the `from` location into the `to` location -pub async fn copy_table( - from: impl AsRef, - from_options: Option>, - to: impl AsRef, - to_options: Option>, - allow_http: bool, -) -> Result<(), DeltaTableError> { - let from_store = DeltaTableBuilder::from_uri(from) - .with_storage_options(from_options.unwrap_or_default()) - .with_allow_http(allow_http) - .build_storage()?; - let to_store = DeltaTableBuilder::from_uri(to) - .with_storage_options(to_options.unwrap_or_default()) - .with_allow_http(allow_http) - .build_storage()?; - sync_stores(from_store.object_store(), to_store.object_store()).await -} - -/// Synchronize the contents of two object stores -pub async fn sync_stores( - from_store: Arc, - to_store: Arc, -) -> Result<(), DeltaTableError> { - // TODO if a table is copied within the same root store (i.e bucket), using copy would be MUCH more efficient - let mut meta_stream = from_store.list(None).await?; - while let Some(file) = meta_stream.next().await { - if let Ok(meta) = file { - let bytes = from_store.get(&meta.location).await?.bytes().await?; - to_store.put(&meta.location, bytes).await?; - } - } - Ok(()) -} /// Collect list stream pub async fn flatten_list_stream( @@ -54,20 +15,11 @@ pub async fn flatten_list_stream( ) -> ObjectStoreResult> { storage .list(prefix) - .await? .map_ok(|meta| meta.location) .try_collect::>() .await } -pub(crate) fn str_is_truthy(val: &str) -> bool { - val.eq_ignore_ascii_case("1") - | val.eq_ignore_ascii_case("true") - | val.eq_ignore_ascii_case("on") - | val.eq_ignore_ascii_case("yes") - | val.eq_ignore_ascii_case("y") -} - impl TryFrom for ObjectMeta { type Error = DeltaTableError; @@ -94,6 +46,7 @@ impl TryFrom<&Add> for ObjectMeta { last_modified, size: value.size as usize, e_tag: None, + version: None, }) } } @@ -116,7 +69,6 @@ mod tests { base_row_id: None, default_row_commit_version: None, deletion_vector: None, - partition_values_parsed: None, stats_parsed: None, clustering_provider: None, }; diff --git a/crates/deltalake-core/src/table/builder.rs b/crates/core/src/table/builder.rs similarity index 65% rename from crates/deltalake-core/src/table/builder.rs rename to crates/core/src/table/builder.rs index b47411383f..2112585836 100644 --- a/crates/deltalake-core/src/table/builder.rs +++ b/crates/core/src/table/builder.rs @@ -7,12 +7,13 @@ use std::sync::Arc; use chrono::{DateTime, FixedOffset, Utc}; use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; +use tracing::debug; use url::Url; use super::DeltaTable; use crate::errors::{DeltaResult, DeltaTableError}; use crate::logstore::LogStoreRef; -use crate::storage::config::{self, StorageOptions}; +use crate::storage::{factories, StorageOptions}; #[allow(dead_code)] #[derive(Debug, thiserror::Error)] @@ -50,7 +51,7 @@ pub enum DeltaVersion { } /// Configuration options for delta table -#[derive(Debug, Serialize, Deserialize)] +#[derive(Debug, Serialize, Deserialize, Clone, PartialEq)] #[serde(rename_all = "camelCase")] pub struct DeltaTableConfig { /// Indicates whether our use case requires tracking tombstones. @@ -75,6 +76,9 @@ pub struct DeltaTableConfig { /// last checkpoint, but will also increase memory usage. Possible rate limits of the storage backend should /// also be considered for optimal performance. pub log_buffer_size: usize, + /// Control the number of records to read / process from the commit / checkpoint files + /// when processing record batches. + pub log_batch_size: usize, } impl Default for DeltaTableConfig { @@ -83,6 +87,7 @@ impl Default for DeltaTableConfig { require_tombstones: true, require_files: true, log_buffer_size: num_cpus::get() * 4, + log_batch_size: 1024, } } } @@ -118,6 +123,9 @@ pub struct DeltaTableLoadOptions { /// last checkpoint, but will also increase memory usage. Possible rate limits of the storage backend should /// also be considered for optimal performance. pub log_buffer_size: usize, + /// Control the number of records to read / process from the commit / checkpoint files + /// when processing record batches. + pub log_batch_size: usize, } impl DeltaTableLoadOptions { @@ -130,15 +138,11 @@ impl DeltaTableLoadOptions { require_files: true, log_buffer_size: num_cpus::get() * 4, version: DeltaVersion::default(), + log_batch_size: 1024, } } } -enum UriType { - LocalPath(PathBuf), - Url(Url), -} - /// builder for configuring a delta table load. #[derive(Debug)] pub struct DeltaTableBuilder { @@ -150,31 +154,48 @@ pub struct DeltaTableBuilder { impl DeltaTableBuilder { /// Creates `DeltaTableBuilder` from table uri + /// + /// Can panic on an invalid URI + /// + /// ```rust + /// # use deltalake_core::table::builder::*; + /// let builder = DeltaTableBuilder::from_uri("../test/tests/data/delta-0.8.0"); + /// assert!(true); + /// ``` pub fn from_uri(table_uri: impl AsRef) -> Self { - Self { - options: DeltaTableLoadOptions::new(table_uri.as_ref()), - storage_options: None, - allow_http: None, - } + let url = ensure_table_uri(&table_uri).expect("The specified table_uri is not valid"); + DeltaTableBuilder::from_valid_uri(url).expect("Failed to create valid builder") } /// Creates `DeltaTableBuilder` from verified table uri. - /// Will fail fast if specified `table_uri` is a local path but doesn't exist. + /// + /// ```rust + /// # use deltalake_core::table::builder::*; + /// let builder = DeltaTableBuilder::from_valid_uri("memory:///"); + /// assert!(builder.is_ok(), "Builder failed with {builder:?}"); + /// ``` pub fn from_valid_uri(table_uri: impl AsRef) -> DeltaResult { - let table_uri = table_uri.as_ref(); - - if let UriType::LocalPath(path) = resolve_uri_type(table_uri)? { - if !path.exists() { - let msg = format!( - "Local path \"{}\" does not exist or you don't have access!", - table_uri - ); - return Err(DeltaTableError::InvalidTableLocation(msg)); + if let Ok(url) = Url::parse(table_uri.as_ref()) { + if url.scheme() == "file" { + let path = url.to_file_path().map_err(|_| { + DeltaTableError::InvalidTableLocation(table_uri.as_ref().to_string()) + })?; + ensure_file_location_exists(path)?; } + } else { + ensure_file_location_exists(PathBuf::from(table_uri.as_ref()))?; } - Ok(DeltaTableBuilder::from_uri(table_uri)) + let url = ensure_table_uri(&table_uri)?; + debug!("creating table builder with {url}"); + + Ok(Self { + options: DeltaTableLoadOptions::new(url), + storage_options: None, + allow_http: None, + }) } + /// Sets `require_tombstones=false` to the builder pub fn without_tombstones(mut self) -> Self { self.options.require_tombstones = false; @@ -266,11 +287,22 @@ impl DeltaTableBuilder { /// Build a delta storage backend for the given config pub fn build_storage(self) -> DeltaResult { - config::configure_log_store( - &self.options.table_uri, - self.storage_options(), - self.options.storage_backend, - ) + debug!("build_storage() with {}", &self.options.table_uri); + let location = Url::parse(&self.options.table_uri).map_err(|_| { + DeltaTableError::NotATable(format!( + "Could not turn {} into a URL", + self.options.table_uri + )) + })?; + + if let Some((store, _url)) = self.options.storage_backend.as_ref() { + debug!("Loading a logstore with a custom store: {store:?}"); + crate::logstore::logstore_with(store.clone(), location, self.storage_options()) + } else { + // If there has been no backend defined just default to the normal logstore look up + debug!("Loading a logstore based off the location: {location:?}"); + crate::logstore::logstore_for(location, self.storage_options()) + } } /// Build the [`DeltaTable`] from specified options. @@ -282,6 +314,7 @@ impl DeltaTableBuilder { require_tombstones: self.options.require_tombstones, require_files: self.options.require_files, log_buffer_size: self.options.log_buffer_size, + log_batch_size: self.options.log_batch_size, }; Ok(DeltaTable::new(self.build_storage()?, config)) } @@ -299,102 +332,9 @@ impl DeltaTableBuilder { } } -/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. -/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. -/// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. -pub mod s3_storage_options { - /// Custom S3 endpoint. - pub const AWS_ENDPOINT_URL: &str = "AWS_ENDPOINT_URL"; - /// The AWS region. - pub const AWS_REGION: &str = "AWS_REGION"; - /// The AWS profile. - pub const AWS_PROFILE: &str = "AWS_PROFILE"; - /// The AWS_ACCESS_KEY_ID to use for S3. - pub const AWS_ACCESS_KEY_ID: &str = "AWS_ACCESS_KEY_ID"; - /// The AWS_SECRET_ACCESS_KEY to use for S3. - pub const AWS_SECRET_ACCESS_KEY: &str = "AWS_SECRET_ACCESS_KEY"; - /// The AWS_SESSION_TOKEN to use for S3. - pub const AWS_SESSION_TOKEN: &str = "AWS_SESSION_TOKEN"; - /// Uses either "path" (the default) or "virtual", which turns on - /// [virtual host addressing](http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). - pub const AWS_S3_ADDRESSING_STYLE: &str = "AWS_S3_ADDRESSING_STYLE"; - /// Locking provider to use for safe atomic rename. - /// `dynamodb` is currently the only supported locking provider. - /// If not set, safe atomic rename is not available. - pub const AWS_S3_LOCKING_PROVIDER: &str = "AWS_S3_LOCKING_PROVIDER"; - /// The role to assume for S3 writes. - pub const AWS_S3_ASSUME_ROLE_ARN: &str = "AWS_S3_ASSUME_ROLE_ARN"; - /// The role session name to use when a role is assumed. If not provided a random session name is generated. - pub const AWS_S3_ROLE_SESSION_NAME: &str = "AWS_S3_ROLE_SESSION_NAME"; - /// The `pool_idle_timeout` option of aws http client. Has to be lower than 20 seconds, which is - /// default S3 server timeout . - /// However, since rusoto uses hyper as a client, its default timeout is 90 seconds - /// . - /// Hence, the `connection closed before message completed` could occur. - /// To avoid that, the default value of this setting is 15 seconds if it's not set otherwise. - pub const AWS_S3_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_S3_POOL_IDLE_TIMEOUT_SECONDS"; - /// The `pool_idle_timeout` for the as3_storage_options sts client. See - /// the reasoning in `AWS_S3_POOL_IDLE_TIMEOUT_SECONDS`. - pub const AWS_STS_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_STS_POOL_IDLE_TIMEOUT_SECONDS"; - /// The number of retries for S3 GET requests failed with 500 Internal Server Error. - pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = - "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES"; - /// The web identity token file to use when using a web identity provider. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; - /// The role name to use for web identity. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; - /// The role session name to use for web identity. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; - /// Allow http connections - mainly useful for integration tests - pub const AWS_ALLOW_HTTP: &str = "AWS_ALLOW_HTTP"; - - /// If set to "true", allows creating commits without concurrent writer protection. - /// Only safe if there is one writer to a given table. - pub const AWS_S3_ALLOW_UNSAFE_RENAME: &str = "AWS_S3_ALLOW_UNSAFE_RENAME"; - - /// The list of option keys owned by the S3 module. - /// Option keys not contained in this list will be added to the `extra_opts` - /// field of [crate::storage::s3::S3StorageOptions]. - pub const S3_OPTS: &[&str] = &[ - AWS_ENDPOINT_URL, - AWS_REGION, - AWS_PROFILE, - AWS_ACCESS_KEY_ID, - AWS_SECRET_ACCESS_KEY, - AWS_SESSION_TOKEN, - AWS_S3_LOCKING_PROVIDER, - AWS_S3_ASSUME_ROLE_ARN, - AWS_S3_ROLE_SESSION_NAME, - AWS_WEB_IDENTITY_TOKEN_FILE, - AWS_ROLE_ARN, - AWS_ROLE_SESSION_NAME, - AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, - AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, - AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - ]; -} - -#[allow(dead_code)] -pub(crate) fn str_option(map: &HashMap, key: &str) -> Option { - map.get(key) - .map_or_else(|| std::env::var(key).ok(), |v| Some(v.to_owned())) -} - -lazy_static::lazy_static! { - static ref KNOWN_SCHEMES: Vec<&'static str> = - Vec::from([ - "file", "memory", "az", "abfs", "abfss", "azure", "wasb", "wasbs", "adl", "s3", "s3a", - "gs", "hdfs", "https", "http", - ]); +enum UriType { + LocalPath(PathBuf), + Url(Url), } /// Utility function to figure out whether string representation of the path @@ -403,18 +343,30 @@ lazy_static::lazy_static! { /// Will return an error if the path is not valid. fn resolve_uri_type(table_uri: impl AsRef) -> DeltaResult { let table_uri = table_uri.as_ref(); + let known_schemes: Vec<_> = factories() + .iter() + .map(|v| v.key().scheme().to_owned()) + .collect(); if let Ok(url) = Url::parse(table_uri) { + let scheme = url.scheme().to_string(); if url.scheme() == "file" { Ok(UriType::LocalPath(url.to_file_path().map_err(|err| { let msg = format!("Invalid table location: {}\nError: {:?}", table_uri, err); DeltaTableError::InvalidTableLocation(msg) })?)) // NOTE this check is required to support absolute windows paths which may properly parse as url - } else if KNOWN_SCHEMES.contains(&url.scheme()) { + } else if known_schemes.contains(&scheme) { Ok(UriType::Url(url)) - } else { + // NOTE this check is required to support absolute windows paths which may properly parse as url + // we assume here that a single character scheme is a windows drive letter + } else if scheme.len() == 1 { Ok(UriType::LocalPath(PathBuf::from(table_uri))) + } else { + Err(DeltaTableError::InvalidTableLocation(format!( + "Unknown scheme: {}", + scheme + ))) } } else { Ok(UriType::LocalPath(PathBuf::from(table_uri))) @@ -470,21 +422,45 @@ pub fn ensure_table_uri(table_uri: impl AsRef) -> DeltaResult { Ok(url) } +/// Validate that the given [PathBuf] does exist, otherwise return a +/// [DeltaTableError::InvalidTableLocation] +fn ensure_file_location_exists(path: PathBuf) -> DeltaResult<()> { + if !path.exists() { + let msg = format!( + "Local path \"{}\" does not exist or you don't have access!", + path.as_path().display(), + ); + return Err(DeltaTableError::InvalidTableLocation(msg)); + } + Ok(()) +} + #[cfg(test)] mod tests { use super::*; - use itertools::Itertools; - use object_store::path::Path; + use crate::storage::DefaultObjectStoreFactory; #[test] fn test_ensure_table_uri() { + factories().insert( + Url::parse("s3://").unwrap(), + Arc::new(DefaultObjectStoreFactory::default()), + ); + // parse an existing relative directory let uri = ensure_table_uri("."); assert!(uri.is_ok()); - let _uri = ensure_table_uri("./nonexistent"); - assert!(uri.is_ok()); let uri = ensure_table_uri("s3://container/path"); assert!(uri.is_ok()); + #[cfg(not(windows))] + { + let uri = ensure_table_uri("file:///tmp/nonexistent/some/path"); + assert!(uri.is_ok()); + } + let uri = ensure_table_uri("./nonexistent"); + assert!(uri.is_ok()); + let file_path = std::path::Path::new("./nonexistent"); + std::fs::remove_dir(file_path).unwrap(); // These cases should all roundtrip to themselves cfg_if::cfg_if! { @@ -541,7 +517,7 @@ mod tests { #[test] fn test_ensure_table_uri_path() { - let tmp_dir = tempdir::TempDir::new("test").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let tmp_path = std::fs::canonicalize(tmp_dir.path()).unwrap(); let paths = &[ tmp_path.join("data/delta-0.8.0"), @@ -561,17 +537,17 @@ mod tests { assert!(!relative_path.exists()); ensure_table_uri(relative_path.as_os_str().to_str().unwrap()).unwrap(); assert!(relative_path.exists()); - std::fs::remove_dir_all(relative_path).unwrap(); + std::fs::remove_dir_all(std::path::Path::new("_tmp")).unwrap(); } #[test] fn test_ensure_table_uri_url() { // Urls should round trips as-is - let expected = Url::parse("s3://tests/data/delta-0.8.0").unwrap(); + let expected = Url::parse("memory:///test/tests/data/delta-0.8.0").unwrap(); let url = ensure_table_uri(&expected).unwrap(); assert_eq!(expected, url); - let tmp_dir = tempdir::TempDir::new("test").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let tmp_path = std::fs::canonicalize(tmp_dir.path()).unwrap(); let path = tmp_path.join("data/delta-0.8.0"); let expected = Url::from_directory_path(path).unwrap(); @@ -579,64 +555,10 @@ mod tests { assert_eq!(expected.as_str().trim_end_matches('/'), url.as_str()); } - #[tokio::test] - async fn read_delta_table_ignoring_tombstones() { - let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") - .without_tombstones() - .load() - .await - .unwrap(); - assert!( - table.get_state().all_tombstones().is_empty(), - "loading without tombstones should skip tombstones" - ); - - assert_eq!( - table.get_files_iter().collect_vec(), - vec![ - Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), - Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet") - ] - ); - } - - #[tokio::test] - async fn read_delta_table_ignoring_files() { - let table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") - .without_files() - .load() - .await - .unwrap(); - - assert_eq!(table.get_files_iter().count(), 0, "files should be empty"); - assert!( - table.get_tombstones().next().is_none(), - "tombstones should be empty" - ); - } - - #[tokio::test] - async fn read_delta_table_with_ignoring_files_on_apply_log() { - let mut table = DeltaTableBuilder::from_uri("./tests/data/delta-0.8.0") - .with_version(0) - .without_files() - .load() - .await - .unwrap(); - - assert_eq!(table.version(), 0); - assert_eq!(table.get_files_iter().count(), 0, "files should be empty"); - assert!( - table.get_tombstones().next().is_none(), - "tombstones should be empty" - ); - - table.update().await.unwrap(); - assert_eq!(table.version(), 1); - assert_eq!(table.get_files_iter().count(), 0, "files should be empty"); - assert!( - table.get_tombstones().next().is_none(), - "tombstones should be empty" - ); + #[test] + fn test_invalid_uri() { + // Urls should round trips as-is + DeltaTableBuilder::from_valid_uri("this://is.nonsense") + .expect_err("this should be an error"); } } diff --git a/crates/deltalake-core/src/table/config.rs b/crates/core/src/table/config.rs similarity index 83% rename from crates/deltalake-core/src/table/config.rs rename to crates/core/src/table/config.rs index 79130de028..ff33c1597b 100644 --- a/crates/deltalake-core/src/table/config.rs +++ b/crates/core/src/table/config.rs @@ -7,6 +7,8 @@ use serde::{Deserialize, Serialize}; use crate::errors::DeltaTableError; +use super::Constraint; + /// Typed property keys that can be defined on a delta table /// /// @@ -45,6 +47,11 @@ pub enum DeltaConfigKey { /// statistics beyond this number, even when such statistics exist). DataSkippingNumIndexedCols, + /// A comma-separated list of column names on which Delta Lake collects statistics to enhance + /// data skipping functionality. This property takes precedence over + /// [DataSkippingNumIndexedCols](Self::DataSkippingNumIndexedCols). + DataSkippingStatsColumns, + /// The shortest duration for Delta Lake to keep logically deleted data files before deleting /// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. /// @@ -59,6 +66,9 @@ pub enum DeltaConfigKey { /// true to enable change data feed. EnableChangeDataFeed, + /// true to enable deletion vectors and predictive I/O for updates. + EnableDeletionVectors, + /// The degree to which a transaction must be isolated from modifications made by concurrent transactions. /// /// Valid values are `Serializable` and `WriteSerializable`. @@ -118,8 +128,10 @@ impl AsRef for DeltaConfigKey { Self::CheckpointPolicy => "delta.checkpointPolicy", Self::ColumnMappingMode => "delta.columnMapping.mode", Self::DataSkippingNumIndexedCols => "delta.dataSkippingNumIndexedCols", + Self::DataSkippingStatsColumns => "delta.dataSkippingStatsColumns", Self::DeletedFileRetentionDuration => "delta.deletedFileRetentionDuration", Self::EnableChangeDataFeed => "delta.enableChangeDataFeed", + Self::EnableDeletionVectors => "delta.enableDeletionVectors", Self::IsolationLevel => "delta.isolationLevel", Self::LogRetentionDuration => "delta.logRetentionDuration", Self::EnableExpiredLogCleanup => "delta.enableExpiredLogCleanup", @@ -148,10 +160,12 @@ impl FromStr for DeltaConfigKey { "delta.checkpointPolicy" => Ok(Self::CheckpointPolicy), "delta.columnMapping.mode" => Ok(Self::ColumnMappingMode), "delta.dataSkippingNumIndexedCols" => Ok(Self::DataSkippingNumIndexedCols), + "delta.dataSkippingStatsColumns" => Ok(Self::DataSkippingStatsColumns), "delta.deletedFileRetentionDuration" | "deletedFileRetentionDuration" => { Ok(Self::DeletedFileRetentionDuration) } "delta.enableChangeDataFeed" => Ok(Self::EnableChangeDataFeed), + "delta.enableDeletionVectors" => Ok(Self::EnableDeletionVectors), "delta.isolationLevel" => Ok(Self::IsolationLevel), "delta.logRetentionDuration" | "logRetentionDuration" => Ok(Self::LogRetentionDuration), "delta.enableExpiredLogCleanup" | "enableExpiredLogCleanup" => { @@ -178,9 +192,9 @@ pub enum DeltaConfigError { } macro_rules! table_config { - ($(($key:expr, $name:ident, $ret:ty, $default:literal),)*) => { + ($(($docs:literal, $key:expr, $name:ident, $ret:ty, $default:literal),)*) => { $( - /// read property $key + #[doc = $docs] pub fn $name(&self) -> $ret { self.0 .get($key.as_ref()) @@ -196,20 +210,29 @@ pub struct TableConfig<'a>(pub(crate) &'a HashMap>); impl<'a> TableConfig<'a> { table_config!( - (DeltaConfigKey::AppendOnly, append_only, bool, false), ( + "true for this Delta table to be append-only", + DeltaConfigKey::AppendOnly, + append_only, + bool, + false + ), + ( + "true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column.", DeltaConfigKey::CheckpointWriteStatsAsJson, write_stats_as_json, bool, true ), ( + "true for Delta Lake to write file statistics to checkpoints in struct format", DeltaConfigKey::CheckpointWriteStatsAsStruct, write_stats_as_struct, bool, - true + false ), ( + "The target file size in bytes or higher units for file tuning", DeltaConfigKey::TargetFileSize, target_file_size, i64, @@ -217,24 +240,37 @@ impl<'a> TableConfig<'a> { 104857600 ), ( + "true to enable change data feed.", DeltaConfigKey::EnableChangeDataFeed, enable_change_data_feed, bool, false ), ( + "true to enable deletion vectors and predictive I/O for updates.", + DeltaConfigKey::EnableDeletionVectors, + enable_deletion_vectors, + bool, + // in databricks the default is dependent on the workspace settings and runtime version + // https://learn.microsoft.com/en-us/azure/databricks/administration-guide/workspace-settings/deletion-vectors + false + ), + ( + "The number of columns for Delta Lake to collect statistics about for data skipping.", DeltaConfigKey::DataSkippingNumIndexedCols, num_indexed_cols, i32, 32 ), ( + "whether to cleanup expired logs", DeltaConfigKey::EnableExpiredLogCleanup, enable_expired_log_cleanup, bool, true ), ( + "Interval (number of commits) after which a new checkpoint should be created", DeltaConfigKey::CheckpointInterval, checkpoint_interval, i32, @@ -253,7 +289,7 @@ impl<'a> TableConfig<'a> { /// than this value. Otherwise, the query may not be able to restart, as it must still read old files. pub fn deleted_file_retention_duration(&self) -> Duration { lazy_static! { - static ref DEFAULT_DURATION: Duration = parse_interval("interval 1 week").unwrap(); + static ref DEFAULT_DURATION: Duration = parse_interval("interval 1 weeks").unwrap(); } self.0 .get(DeltaConfigKey::DeletedFileRetentionDuration.as_ref()) @@ -269,7 +305,7 @@ impl<'a> TableConfig<'a> { /// constant time. Operations on history are parallel but will become more expensive as the log size increases. pub fn log_retention_duration(&self) -> Duration { lazy_static! { - static ref DEFAULT_DURATION: Duration = parse_interval("interval 30 day").unwrap(); + static ref DEFAULT_DURATION: Duration = parse_interval("interval 30 days").unwrap(); } self.0 .get(DeltaConfigKey::LogRetentionDuration.as_ref()) @@ -302,6 +338,28 @@ impl<'a> TableConfig<'a> { .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) .unwrap_or_default() } + + /// Return the check constraints on the current table + pub fn get_constraints(&self) -> Vec { + self.0 + .iter() + .filter_map(|(field, value)| { + if field.starts_with("delta.constraints") { + value.as_ref().map(|f| Constraint::new("*", f)) + } else { + None + } + }) + .collect() + } + + /// Column names on which Delta Lake collects statistics to enhance data skipping functionality. + /// This property takes precedence over [num_indexed_cols](Self::num_indexed_cols). + pub fn stats_columns(&self) -> Option> { + self.0 + .get(DeltaConfigKey::DataSkippingStatsColumns.as_ref()) + .and_then(|o| o.as_ref().map(|v| v.split(',').collect())) + } } #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] @@ -467,14 +525,14 @@ fn parse_interval(value: &str) -> Result { let number = number as u64; let duration = match it.next().ok_or_else(not_an_interval)? { - "nanosecond" => Duration::from_nanos(number), - "microsecond" => Duration::from_micros(number), - "millisecond" => Duration::from_millis(number), - "second" => Duration::from_secs(number), - "minute" => Duration::from_secs(number * SECONDS_PER_MINUTE), - "hour" => Duration::from_secs(number * SECONDS_PER_HOUR), - "day" => Duration::from_secs(number * SECONDS_PER_DAY), - "week" => Duration::from_secs(number * SECONDS_PER_WEEK), + "nanosecond" | "nanoseconds" => Duration::from_nanos(number), + "microsecond" | "microseconds" => Duration::from_micros(number), + "millisecond" | "milliseconds" => Duration::from_millis(number), + "second" | "seconds" => Duration::from_secs(number), + "minute" | "minutes" => Duration::from_secs(number * SECONDS_PER_MINUTE), + "hour" | "hours" => Duration::from_secs(number * SECONDS_PER_HOUR), + "day" | "days" => Duration::from_secs(number * SECONDS_PER_DAY), + "week" | "weeks" => Duration::from_secs(number * SECONDS_PER_WEEK), unit => { return Err(DeltaConfigError::Validation(format!( "Unknown unit '{unit}'" @@ -494,13 +552,12 @@ fn parse_int(value: &str) -> Result { #[cfg(test)] mod tests { use super::*; - use crate::kernel::StructType; - use crate::table::DeltaTableMetaData; + use crate::kernel::{Metadata, StructType}; use std::collections::HashMap; - fn dummy_metadata() -> DeltaTableMetaData { + fn dummy_metadata() -> Metadata { let schema = StructType::new(Vec::new()); - DeltaTableMetaData::new(None, None, None, schema, Vec::new(), HashMap::new()) + Metadata::try_new(schema, Vec::::new(), HashMap::new()).unwrap() } #[test] @@ -563,36 +620,76 @@ mod tests { Duration::from_nanos(123) ); + assert_eq!( + parse_interval("interval 123 nanoseconds").unwrap(), + Duration::from_nanos(123) + ); + assert_eq!( parse_interval("interval 123 microsecond").unwrap(), Duration::from_micros(123) ); + assert_eq!( + parse_interval("interval 123 microseconds").unwrap(), + Duration::from_micros(123) + ); + assert_eq!( parse_interval("interval 123 millisecond").unwrap(), Duration::from_millis(123) ); + assert_eq!( + parse_interval("interval 123 milliseconds").unwrap(), + Duration::from_millis(123) + ); + assert_eq!( parse_interval("interval 123 second").unwrap(), Duration::from_secs(123) ); + assert_eq!( + parse_interval("interval 123 seconds").unwrap(), + Duration::from_secs(123) + ); + assert_eq!( parse_interval("interval 123 minute").unwrap(), Duration::from_secs(123 * 60) ); + assert_eq!( + parse_interval("interval 123 minutes").unwrap(), + Duration::from_secs(123 * 60) + ); + assert_eq!( parse_interval("interval 123 hour").unwrap(), Duration::from_secs(123 * 3600) ); + assert_eq!( + parse_interval("interval 123 hours").unwrap(), + Duration::from_secs(123 * 3600) + ); + assert_eq!( parse_interval("interval 123 day").unwrap(), Duration::from_secs(123 * 86400) ); + assert_eq!( + parse_interval("interval 123 days").unwrap(), + Duration::from_secs(123 * 86400) + ); + + assert_eq!( + parse_interval("interval 123 week").unwrap(), + Duration::from_secs(123 * 604800) + ); + assert_eq!( parse_interval("interval 123 week").unwrap(), Duration::from_secs(123 * 604800) diff --git a/crates/core/src/table/mod.rs b/crates/core/src/table/mod.rs new file mode 100644 index 0000000000..becebf0501 --- /dev/null +++ b/crates/core/src/table/mod.rs @@ -0,0 +1,605 @@ +//! Delta Table read and write implementation + +use std::cmp::Ordering; +use std::collections::HashMap; +use std::fmt; +use std::fmt::Formatter; + +use chrono::{DateTime, Utc}; +use futures::TryStreamExt; +use object_store::{path::Path, ObjectStore}; +use serde::de::{Error, SeqAccess, Visitor}; +use serde::ser::SerializeSeq; +use serde::{Deserialize, Deserializer, Serialize, Serializer}; +use tracing::debug; + +use self::builder::DeltaTableConfig; +use self::state::DeltaTableState; +use crate::kernel::{ + Action, CommitInfo, DataCheck, DataType, LogicalFile, Metadata, Protocol, StructType, +}; +use crate::logstore::{self, LogStoreConfig, LogStoreRef}; +use crate::partitions::PartitionFilter; +use crate::storage::{commit_uri_from_version, ObjectStoreRef}; +use crate::{DeltaResult, DeltaTableError}; + +pub mod builder; +pub mod config; +pub mod state; +pub mod state_arrow; + +/// Metadata for a checkpoint file +#[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] +pub struct CheckPoint { + /// Delta table version + pub(crate) version: i64, // 20 digits decimals + /// The number of actions that are stored in the checkpoint. + pub(crate) size: i64, + #[serde(skip_serializing_if = "Option::is_none")] + /// The number of fragments if the last checkpoint was written in multiple parts. This field is optional. + pub(crate) parts: Option, // 10 digits decimals + #[serde(skip_serializing_if = "Option::is_none")] + /// The number of bytes of the checkpoint. This field is optional. + pub(crate) size_in_bytes: Option, + #[serde(skip_serializing_if = "Option::is_none")] + /// The number of AddFile actions in the checkpoint. This field is optional. + pub(crate) num_of_add_files: Option, +} + +#[derive(Default)] +/// Builder for CheckPoint +pub struct CheckPointBuilder { + /// Delta table version + pub(crate) version: i64, // 20 digits decimals + /// The number of actions that are stored in the checkpoint. + pub(crate) size: i64, + /// The number of fragments if the last checkpoint was written in multiple parts. This field is optional. + pub(crate) parts: Option, // 10 digits decimals + /// The number of bytes of the checkpoint. This field is optional. + pub(crate) size_in_bytes: Option, + /// The number of AddFile actions in the checkpoint. This field is optional. + pub(crate) num_of_add_files: Option, +} + +impl CheckPointBuilder { + /// Creates a new [`CheckPointBuilder`] instance with the provided `version` and `size`. + /// Size is the total number of actions in the checkpoint. See size_in_bytes for total size in bytes. + pub fn new(version: i64, size: i64) -> Self { + CheckPointBuilder { + version, + size, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + } + } + + /// The number of fragments if the last checkpoint was written in multiple parts. This field is optional. + pub fn with_parts(mut self, parts: u32) -> Self { + self.parts = Some(parts); + self + } + + /// The number of bytes of the checkpoint. This field is optional. + pub fn with_size_in_bytes(mut self, size_in_bytes: i64) -> Self { + self.size_in_bytes = Some(size_in_bytes); + self + } + + /// The number of AddFile actions in the checkpoint. This field is optional. + pub fn with_num_of_add_files(mut self, num_of_add_files: i64) -> Self { + self.num_of_add_files = Some(num_of_add_files); + self + } + + /// Build the final [`CheckPoint`] struct. + pub fn build(self) -> CheckPoint { + CheckPoint { + version: self.version, + size: self.size, + parts: self.parts, + size_in_bytes: self.size_in_bytes, + num_of_add_files: self.num_of_add_files, + } + } +} + +impl CheckPoint { + /// Creates a new checkpoint from the given parameters. + pub fn new(version: i64, size: i64, parts: Option) -> Self { + Self { + version, + size, + parts: parts.or(None), + size_in_bytes: None, + num_of_add_files: None, + } + } +} + +impl PartialEq for CheckPoint { + fn eq(&self, other: &Self) -> bool { + self.version == other.version + } +} + +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. + 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(), + } + } +} + +impl DataCheck for Constraint { + fn get_name(&self) -> &str { + &self.name + } + + fn get_expression(&self) -> &str { + &self.expr + } +} + +/// Return partition fields along with their data type from the current schema. +pub(crate) fn get_partition_col_data_types<'a>( + schema: &'a StructType, + metadata: &'a Metadata, +) -> Vec<(&'a String, &'a DataType)> { + // JSON add actions contain a `partitionValues` field which is a map. + // When loading `partitionValues_parsed` we have to convert the stringified partition values back to the correct data type. + schema + .fields() + .iter() + .filter_map(|f| { + if metadata + .partition_columns + .iter() + .any(|s| s.as_str() == f.name()) + { + Some((f.name(), f.data_type())) + } else { + None + } + }) + .collect() +} + +/// The next commit that's available from underlying storage +/// TODO: Maybe remove this and replace it with Some/None and create a `Commit` struct to contain the next commit +/// +#[derive(Debug)] +pub enum PeekCommit { + /// The next commit version and associated actions + New(i64, Vec), + /// Provided DeltaVersion is up to date + UpToDate, +} + +/// In memory representation of a Delta Table +#[derive(Clone)] +pub struct DeltaTable { + /// The state of the table as of the most recent loaded Delta log entry. + pub state: Option, + /// the load options used during load + pub config: DeltaTableConfig, + /// log store + pub(crate) log_store: LogStoreRef, +} + +impl Serialize for DeltaTable { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + let mut seq = serializer.serialize_seq(None)?; + seq.serialize_element(&self.state)?; + seq.serialize_element(&self.config)?; + seq.serialize_element(self.log_store.config())?; + seq.end() + } +} + +impl<'de> Deserialize<'de> for DeltaTable { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + struct DeltaTableVisitor {} + + impl<'de> Visitor<'de> for DeltaTableVisitor { + type Value = DeltaTable; + + fn expecting(&self, formatter: &mut Formatter) -> fmt::Result { + formatter.write_str("struct DeltaTable") + } + + fn visit_seq(self, mut seq: A) -> Result + where + A: SeqAccess<'de>, + { + let state = seq + .next_element()? + .ok_or_else(|| A::Error::invalid_length(0, &self))?; + let config = seq + .next_element()? + .ok_or_else(|| A::Error::invalid_length(0, &self))?; + let storage_config: LogStoreConfig = seq + .next_element()? + .ok_or_else(|| A::Error::invalid_length(0, &self))?; + let log_store = + crate::logstore::logstore_for(storage_config.location, storage_config.options) + .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; + + let table = DeltaTable { + state, + config, + log_store, + }; + Ok(table) + } + } + + deserializer.deserialize_seq(DeltaTableVisitor {}) + } +} + +impl DeltaTable { + /// Create a new Delta Table struct without loading any data from backing storage. + /// + /// NOTE: This is for advanced users. If you don't know why you need to use this method, please + /// call one of the `open_table` helper methods instead. + pub fn new(log_store: LogStoreRef, config: DeltaTableConfig) -> Self { + Self { + state: None, + log_store, + config, + } + } + + /// Create a new [`DeltaTable`] from a [`DeltaTableState`] without loading any + /// data from backing storage. + /// + /// NOTE: This is for advanced users. If you don't know why you need to use this method, + /// please call one of the `open_table` helper methods instead. + pub(crate) fn new_with_state(log_store: LogStoreRef, state: DeltaTableState) -> Self { + Self { + state: Some(state), + log_store, + config: Default::default(), + } + } + + /// get a shared reference to the delta object store + pub fn object_store(&self) -> ObjectStoreRef { + self.log_store.object_store() + } + + /// The URI of the underlying data + pub fn table_uri(&self) -> String { + self.log_store.root_uri() + } + + /// get a shared reference to the log store + pub fn log_store(&self) -> LogStoreRef { + self.log_store.clone() + } + + /// returns the latest available version of the table + pub async fn get_latest_version(&self) -> Result { + self.log_store.get_latest_version(self.version()).await + } + + /// Currently loaded version of the table + pub fn version(&self) -> i64 { + self.state.as_ref().map(|s| s.version()).unwrap_or(-1) + } + + /// Load DeltaTable with data from latest checkpoint + pub async fn load(&mut self) -> Result<(), DeltaTableError> { + self.update_incremental(None).await + } + + /// Updates the DeltaTable to the most recent state committed to the transaction log by + /// loading the last checkpoint and incrementally applying each version since. + pub async fn update(&mut self) -> Result<(), DeltaTableError> { + self.update_incremental(None).await + } + + /// Get the list of actions for the next commit + pub async fn peek_next_commit( + &self, + current_version: i64, + ) -> Result { + let next_version = current_version + 1; + let commit_log_bytes = match self.log_store.read_commit_entry(next_version).await { + Ok(Some(bytes)) => Ok(bytes), + Ok(None) => return Ok(PeekCommit::UpToDate), + Err(err) => Err(err), + }?; + + let actions = logstore::get_actions(next_version, commit_log_bytes).await; + Ok(PeekCommit::New(next_version, actions.unwrap())) + } + + /// Updates the DeltaTable to the latest version by incrementally applying newer versions. + /// It assumes that the table is already updated to the current version `self.version`. + pub async fn update_incremental( + &mut self, + max_version: Option, + ) -> Result<(), DeltaTableError> { + debug!( + "incremental update with version({}) and max_version({max_version:?})", + self.version(), + ); + match self.state.as_mut() { + Some(state) => state.update(self.log_store.clone(), max_version).await, + _ => { + let state = DeltaTableState::try_new( + &Path::default(), + self.log_store.object_store(), + self.config.clone(), + max_version, + ) + .await?; + self.state = Some(state); + Ok(()) + } + } + } + + /// Loads the DeltaTable state for the given version. + pub async fn load_version(&mut self, version: i64) -> Result<(), DeltaTableError> { + if let Some(snapshot) = &self.state { + if snapshot.version() > version { + self.state = None; + } + } + self.update_incremental(Some(version)).await + } + + pub(crate) async fn get_version_timestamp(&self, version: i64) -> Result { + match self + .state + .as_ref() + .and_then(|s| s.version_timestamp(version)) + { + Some(ts) => Ok(ts), + None => { + let meta = self + .object_store() + .head(&commit_uri_from_version(version)) + .await?; + let ts = meta.last_modified.timestamp_millis(); + Ok(ts) + } + } + } + + /// Returns provenance information, including the operation, user, and so on, for each write to a table. + /// The table history retention is based on the `logRetentionDuration` property of the Delta Table, 30 days by default. + /// If `limit` is given, this returns the information of the latest `limit` commits made to this table. Otherwise, + /// it returns all commits from the earliest commit. + pub async fn history(&self, limit: Option) -> Result, DeltaTableError> { + let infos = self + .snapshot()? + .snapshot + .snapshot() + .commit_infos(self.object_store(), limit) + .await? + .try_collect::>() + .await?; + Ok(infos.into_iter().flatten().collect()) + } + + /// Obtain Add actions for files that match the filter + pub fn get_active_add_actions_by_partitions<'a>( + &'a self, + filters: &'a [PartitionFilter], + ) -> Result>> + '_, DeltaTableError> { + self.state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .get_active_add_actions_by_partitions(filters) + } + + /// Returns the file list tracked in current table state filtered by provided + /// `PartitionFilter`s. + pub fn get_files_by_partitions( + &self, + filters: &[PartitionFilter], + ) -> Result, DeltaTableError> { + Ok(self + .get_active_add_actions_by_partitions(filters)? + .collect::, _>>()? + .into_iter() + .map(|add| add.object_store_path()) + .collect()) + } + + /// Return the file uris as strings for the partition(s) + pub fn get_file_uris_by_partitions( + &self, + filters: &[PartitionFilter], + ) -> Result, DeltaTableError> { + let files = self.get_files_by_partitions(filters)?; + Ok(files + .iter() + .map(|fname| self.log_store.to_uri(fname)) + .collect()) + } + + /// Returns an iterator of file names present in the loaded state + #[inline] + pub fn get_files_iter(&self) -> DeltaResult + '_> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .file_paths_iter()) + } + + /// Returns a URIs for all active files present in the current table version. + pub fn get_file_uris(&self) -> DeltaResult + '_> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .file_paths_iter() + .map(|path| self.log_store.to_uri(&path))) + } + + /// Get the number of files in the table - retrn 0 if no metadata is loaded + pub fn get_files_count(&self) -> usize { + self.state.as_ref().map(|s| s.files_count()).unwrap_or(0) + } + + /// Returns the currently loaded state snapshot. + pub fn snapshot(&self) -> DeltaResult<&DeltaTableState> { + self.state.as_ref().ok_or(DeltaTableError::NotInitialized) + } + + /// Returns current table protocol + pub fn protocol(&self) -> DeltaResult<&Protocol> { + Ok(self + .state + .as_ref() + .ok_or(DeltaTableError::NoMetadata)? + .protocol()) + } + + /// Returns the metadata associated with the loaded state. + pub fn metadata(&self) -> Result<&Metadata, DeltaTableError> { + Ok(self.snapshot()?.metadata()) + } + + /// Returns the current version of the DeltaTable based on the loaded metadata. + pub fn get_app_transaction_version(&self) -> HashMap { + self.state + .as_ref() + .map(|s| s.app_transaction_version().clone()) + .unwrap_or_default() + } + + /// Return table schema parsed from transaction log. Return None if table hasn't been loaded or + /// no metadata was found in the log. + pub fn schema(&self) -> Option<&StructType> { + Some(self.snapshot().ok()?.schema()) + } + + /// Return table schema parsed from transaction log. Return `DeltaTableError` if table hasn't + /// been loaded or no metadata was found in the log. + pub fn get_schema(&self) -> Result<&StructType, DeltaTableError> { + Ok(self.snapshot()?.schema()) + } + + /// Time travel Delta table to the latest version that's created at or before provided + /// `datetime` argument. + /// + /// Internally, this methods performs a binary search on all Delta transaction logs. + pub async fn load_with_datetime( + &mut self, + datetime: DateTime, + ) -> Result<(), DeltaTableError> { + let mut min_version = 0; + let mut max_version = self.get_latest_version().await?; + let mut version = min_version; + let target_ts = datetime.timestamp_millis(); + + // binary search + while min_version <= max_version { + let pivot = (max_version + min_version) / 2; + version = pivot; + let pts = self.get_version_timestamp(pivot).await?; + match pts.cmp(&target_ts) { + Ordering::Equal => { + break; + } + Ordering::Less => { + min_version = pivot + 1; + } + Ordering::Greater => { + max_version = pivot - 1; + version = max_version + } + } + } + + if version < 0 { + version = 0; + } + + self.load_version(version).await + } +} + +impl fmt::Display for DeltaTable { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + writeln!(f, "DeltaTable({})", self.table_uri())?; + writeln!(f, "\tversion: {}", self.version()) + } +} + +impl std::fmt::Debug for DeltaTable { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(f, "DeltaTable <{}>", self.table_uri()) + } +} + +#[cfg(test)] +mod tests { + use pretty_assertions::assert_eq; + use tempfile::TempDir; + + use super::*; + use crate::kernel::{DataType, PrimitiveType, StructField}; + use crate::operations::create::CreateBuilder; + + #[tokio::test] + async fn table_round_trip() { + let (dt, tmp_dir) = create_test_table().await; + let bytes = serde_json::to_vec(&dt).unwrap(); + let actual: DeltaTable = serde_json::from_slice(&bytes).unwrap(); + assert_eq!(actual.version(), dt.version()); + drop(tmp_dir); + } + + async fn create_test_table() -> (DeltaTable, TempDir) { + let tmp_dir = tempfile::tempdir().unwrap(); + let table_dir = tmp_dir.path().join("test_create"); + std::fs::create_dir(&table_dir).unwrap(); + + let dt = CreateBuilder::new() + .with_location(table_dir.to_str().unwrap()) + .with_table_name("Test Table Create") + .with_comment("This table is made to test the create function for a DeltaTable") + .with_columns(vec![ + StructField::new( + "Id".to_string(), + DataType::Primitive(PrimitiveType::Integer), + true, + ), + StructField::new( + "Name".to_string(), + DataType::Primitive(PrimitiveType::String), + true, + ), + ]) + .await + .unwrap(); + (dt, tmp_dir) + } +} diff --git a/crates/core/src/table/state.rs b/crates/core/src/table/state.rs new file mode 100644 index 0000000000..89595e870a --- /dev/null +++ b/crates/core/src/table/state.rs @@ -0,0 +1,267 @@ +//! The module for delta table state. + +use std::collections::HashMap; +use std::sync::Arc; + +use chrono::Utc; +use futures::TryStreamExt; +use object_store::{path::Path, ObjectStore}; +use serde::{Deserialize, Serialize}; + +use super::config::TableConfig; +use super::{get_partition_col_data_types, DeltaTableConfig}; +use crate::kernel::{ + Action, Add, DataType, EagerSnapshot, LogDataHandler, LogicalFile, Metadata, Protocol, Remove, + StructType, +}; +use crate::logstore::LogStore; +use crate::operations::transaction::CommitData; +use crate::partitions::{DeltaTablePartition, PartitionFilter}; +use crate::protocol::DeltaOperation; +use crate::{DeltaResult, DeltaTableError}; + +/// State snapshot currently held by the Delta Table instance. +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct DeltaTableState { + app_transaction_version: HashMap, + pub(crate) snapshot: EagerSnapshot, +} + +impl DeltaTableState { + /// Create a new DeltaTableState + pub async fn try_new( + table_root: &Path, + store: Arc, + config: DeltaTableConfig, + version: Option, + ) -> DeltaResult { + let snapshot = EagerSnapshot::try_new(table_root, store.clone(), config, version).await?; + Ok(Self { + snapshot, + app_transaction_version: HashMap::new(), + }) + } + + /// Return table version + pub fn version(&self) -> i64 { + self.snapshot.version() + } + + /// Get the timestamp when a version commit was created. + /// This is the timestamp of the commit file. + /// If the commit file is not present, None is returned. + pub fn version_timestamp(&self, version: i64) -> Option { + self.snapshot.version_timestamp(version) + } + + /// Construct a delta table state object from a list of actions + #[cfg(test)] + pub fn from_actions(actions: Vec) -> DeltaResult { + use crate::protocol::SaveMode; + let metadata = actions + .iter() + .find_map(|a| match a { + Action::Metadata(m) => Some(m.clone()), + _ => None, + }) + .ok_or(DeltaTableError::NotInitialized)?; + let protocol = actions + .iter() + .find_map(|a| match a { + Action::Protocol(p) => Some(p.clone()), + _ => None, + }) + .ok_or(DeltaTableError::NotInitialized)?; + + let commit_data = [CommitData::new( + actions, + DeltaOperation::Create { + mode: SaveMode::Append, + location: Path::default().to_string(), + protocol: protocol.clone(), + metadata: metadata.clone(), + }, + HashMap::new(), + ) + .unwrap()]; + + let snapshot = EagerSnapshot::new_test(&commit_data).unwrap(); + Ok(Self { + app_transaction_version: Default::default(), + snapshot, + }) + } + + /// Returns a semantic accessor to the currently loaded log data. + pub fn log_data(&self) -> LogDataHandler<'_> { + self.snapshot.log_data() + } + + /// Full list of tombstones (remove actions) representing files removed from table state). + pub async fn all_tombstones( + &self, + store: Arc, + ) -> DeltaResult> { + Ok(self + .snapshot + .snapshot() + .tombstones(store)? + .try_collect::>() + .await? + .into_iter() + .flatten()) + } + + /// List of unexpired tombstones (remove actions) representing files removed from table state. + /// The retention period is set by `deletedFileRetentionDuration` with default value of 1 week. + pub async fn unexpired_tombstones( + &self, + store: Arc, + ) -> DeltaResult> { + let retention_timestamp = Utc::now().timestamp_millis() + - self + .table_config() + .deleted_file_retention_duration() + .as_millis() as i64; + let tombstones = self.all_tombstones(store).await?.collect::>(); + Ok(tombstones + .into_iter() + .filter(move |t| t.deletion_timestamp.unwrap_or(0) > retention_timestamp)) + } + + /// Full list of add actions representing all parquet files that are part of the current + /// delta table state. + pub fn file_actions(&self) -> DeltaResult> { + Ok(self.snapshot.file_actions()?.collect()) + } + + /// Get the number of files in the current table state + pub fn files_count(&self) -> usize { + self.snapshot.files_count() + } + + /// Returns an iterator of file names present in the loaded state + #[inline] + pub fn file_paths_iter(&self) -> impl Iterator + '_ { + self.log_data() + .into_iter() + .map(|add| add.object_store_path()) + } + + /// HashMap containing the last txn version stored for every app id writing txn + /// actions. + pub fn app_transaction_version(&self) -> &HashMap { + &self.app_transaction_version + } + + /// The most recent protocol of the table. + pub fn protocol(&self) -> &Protocol { + self.snapshot.protocol() + } + + /// The most recent metadata of the table. + pub fn metadata(&self) -> &Metadata { + self.snapshot.metadata() + } + + /// The table schema + pub fn schema(&self) -> &StructType { + self.snapshot.schema() + } + + /// Well known table configuration + pub fn table_config(&self) -> TableConfig<'_> { + self.snapshot.table_config() + } + + /// Merges new state information into our state + /// + /// The DeltaTableState also carries the version information for the given state, + /// as there is a one-to-one match between a table state and a version. In merge/update + /// scenarios we cannot infer the intended / correct version number. By default this + /// function will update the tracked version if the version on `new_state` is larger then the + /// currently set version however it is up to the caller to update the `version` field according + /// to the version the merged state represents. + pub(crate) fn merge( + &mut self, + actions: Vec, + operation: &DeltaOperation, + version: i64, + ) -> Result<(), DeltaTableError> { + // TODO: Maybe change this interface to just use CommitData.. + let commit_data = CommitData { + actions, + operation: operation.clone(), + app_metadata: HashMap::new(), + }; + let new_version = self.snapshot.advance(&vec![commit_data])?; + if new_version != version { + return Err(DeltaTableError::Generic("Version mismatch".to_string())); + } + Ok(()) + } + + /// Obtain the Eager snapshot of the state + pub fn snapshot(&self) -> &EagerSnapshot { + &self.snapshot + } + + /// Update the state of the table to the given version. + pub async fn update( + &mut self, + log_store: Arc, + version: Option, + ) -> Result<(), DeltaTableError> { + self.snapshot.update(log_store, version).await?; + Ok(()) + } + + /// Obtain Add actions for files that match the filter + pub fn get_active_add_actions_by_partitions<'a>( + &'a self, + filters: &'a [PartitionFilter], + ) -> Result>> + '_, DeltaTableError> { + let current_metadata = self.metadata(); + + let nonpartitioned_columns: Vec = filters + .iter() + .filter(|f| !current_metadata.partition_columns.contains(&f.key)) + .map(|f| f.key.to_string()) + .collect(); + + if !nonpartitioned_columns.is_empty() { + return Err(DeltaTableError::ColumnsNotPartitioned { + nonpartitioned_columns: { nonpartitioned_columns }, + }); + } + + let partition_col_data_types: HashMap<&String, &DataType> = + get_partition_col_data_types(self.schema(), current_metadata) + .into_iter() + .collect(); + + Ok(self.log_data().into_iter().filter_map(move |add| { + let partitions = add.partition_values(); + if partitions.is_err() { + return Some(Err(DeltaTableError::Generic( + "Failed to parse partition values".to_string(), + ))); + } + let partitions = partitions + .unwrap() + .iter() + .map(|(k, v)| DeltaTablePartition::from_partition_value((*k, v))) + .collect::>(); + let is_valid = filters + .iter() + .all(|filter| filter.match_partitions(&partitions, &partition_col_data_types)); + + if is_valid { + Some(Ok(add)) + } else { + None + } + })) + } +} diff --git a/crates/deltalake-core/src/table/state_arrow.rs b/crates/core/src/table/state_arrow.rs similarity index 91% rename from crates/deltalake-core/src/table/state_arrow.rs rename to crates/core/src/table/state_arrow.rs index 5e2565ee08..2c9ed74ce3 100644 --- a/crates/deltalake-core/src/table/state_arrow.rs +++ b/crates/core/src/table/state_arrow.rs @@ -19,7 +19,7 @@ use itertools::Itertools; use super::config::ColumnMappingMode; use super::state::DeltaTableState; use crate::errors::DeltaTableError; -use crate::kernel::{DataType as DeltaDataType, StructType}; +use crate::kernel::{Add, DataType as DeltaDataType, StructType}; use crate::protocol::{ColumnCountStat, ColumnValueStat, Stats}; impl DeltaTableState { @@ -54,27 +54,22 @@ impl DeltaTableState { &self, flatten: bool, ) -> Result { + let files = self.file_actions()?; let mut paths = arrow::array::StringBuilder::with_capacity( - self.files().len(), - self.files().iter().map(|add| add.path.len()).sum(), + files.len(), + files.iter().map(|add| add.path.len()).sum(), ); - for action in self.files() { + for action in &files { paths.append_value(&action.path); } - let size = self - .files() - .iter() - .map(|file| file.size) - .collect::(); - let mod_time: TimestampMillisecondArray = self - .files() + let size = files.iter().map(|file| file.size).collect::(); + let mod_time: TimestampMillisecondArray = files .iter() .map(|file| file.modification_time) .collect::>() .into(); - let data_change = self - .files() + let data_change = files .iter() .map(|file| Some(file.data_change)) .collect::(); @@ -86,10 +81,10 @@ impl DeltaTableState { (Cow::Borrowed("data_change"), Arc::new(data_change)), ]; - let metadata = self.metadata().ok_or(DeltaTableError::NoMetadata)?; + let metadata = self.metadata(); if !metadata.partition_columns.is_empty() { - let partition_cols_batch = self.partition_columns_as_batch(flatten)?; + let partition_cols_batch = self.partition_columns_as_batch(flatten, &files)?; arrays.extend( partition_cols_batch .schema() @@ -100,7 +95,7 @@ impl DeltaTableState { ) } - if self.files().iter().any(|add| add.stats.is_some()) { + if files.iter().any(|add| add.stats.is_some()) { let stats = self.stats_as_batch(flatten)?; arrays.extend( stats @@ -111,8 +106,8 @@ impl DeltaTableState { .zip(stats.columns().iter().map(Arc::clone)), ); } - if self.files().iter().any(|add| add.deletion_vector.is_some()) { - let delvs = self.deletion_vectors_as_batch(flatten)?; + if files.iter().any(|add| add.deletion_vector.is_some()) { + let delvs = self.deletion_vectors_as_batch(flatten, &files)?; arrays.extend( delvs .schema() @@ -122,13 +117,13 @@ impl DeltaTableState { .zip(delvs.columns().iter().map(Arc::clone)), ); } - if self.files().iter().any(|add| { + if files.iter().any(|add| { add.tags .as_ref() .map(|tags| !tags.is_empty()) .unwrap_or(false) }) { - let tags = self.tags_as_batch(flatten)?; + let tags = self.tags_as_batch(flatten, &files)?; arrays.extend( tags.schema() .fields @@ -144,15 +139,17 @@ impl DeltaTableState { fn partition_columns_as_batch( &self, flatten: bool, + files: &Vec, ) -> Result { - let metadata = self.metadata().ok_or(DeltaTableError::NoMetadata)?; + let metadata = self.metadata(); let column_mapping_mode = self.table_config().column_mapping_mode(); let partition_column_types: Vec = metadata .partition_columns .iter() .map( |name| -> Result { - let field = metadata.schema.field_with_name(name)?; + let schema = metadata.schema()?; + let field = schema.field_with_name(name)?; Ok(field.data_type().try_into()?) }, ) @@ -174,21 +171,21 @@ impl DeltaTableState { .partition_columns .iter() .map(|name| -> Result<_, DeltaTableError> { - let physical_name = metadata - .schema + let physical_name = self + .schema() .field_with_name(name) .or(Err(DeltaTableError::MetadataError(format!( "Invalid partition column {0}", name ))))? - .physical_name() - .map_err(|e| DeltaTableError::Kernel { source: e })?; + .physical_name()? + .to_string(); Ok((physical_name, name.as_str())) }) - .collect::, DeltaTableError>>()?, + .collect::, DeltaTableError>>()?, }; // Append values - for action in self.files() { + for action in files { for (name, maybe_value) in action.partition_values.iter() { let logical_name = match column_mapping_mode { ColumnMappingMode::None => name.as_str(), @@ -260,9 +257,9 @@ impl DeltaTableState { fn tags_as_batch( &self, flatten: bool, + files: &Vec, ) -> Result { - let tag_keys: HashSet<&str> = self - .files() + let tag_keys: HashSet<&str> = files .iter() .flat_map(|add| add.tags.as_ref().map(|tags| tags.keys())) .flatten() @@ -273,12 +270,12 @@ impl DeltaTableState { .map(|&key| { ( key, - arrow::array::StringBuilder::with_capacity(self.files().len(), 64), + arrow::array::StringBuilder::with_capacity(files.len(), 64), ) }) .collect(); - for add in self.files() { + for add in files { for &key in &tag_keys { if let Some(value) = add .tags @@ -320,15 +317,16 @@ impl DeltaTableState { fn deletion_vectors_as_batch( &self, flatten: bool, + files: &Vec, ) -> Result { - let capacity = self.files().len(); + let capacity = files.len(); let mut storage_type = arrow::array::StringBuilder::with_capacity(capacity, 1); let mut path_or_inline_div = arrow::array::StringBuilder::with_capacity(capacity, 64); let mut offset = arrow::array::Int32Builder::with_capacity(capacity); let mut size_in_bytes = arrow::array::Int32Builder::with_capacity(capacity); let mut cardinality = arrow::array::Int64Builder::with_capacity(capacity); - for add in self.files() { + for add in files { if let Some(value) = &add.deletion_vector { storage_type.append_value(&value.storage_type); path_or_inline_div.append_value(value.path_or_inline_dv.clone()); @@ -399,7 +397,7 @@ impl DeltaTableState { flatten: bool, ) -> Result { let stats: Vec> = self - .files() + .file_actions()? .iter() .map(|f| { f.get_stats() @@ -413,8 +411,7 @@ impl DeltaTableState { .map(|maybe_stat| maybe_stat.as_ref().map(|stat| stat.num_records)) .collect::>>(), ); - let metadata = self.metadata().ok_or(DeltaTableError::NoMetadata)?; - let schema = &metadata.schema; + let schema = self.schema(); #[derive(Debug)] struct ColStats<'a> { @@ -740,8 +737,8 @@ fn json_value_to_array_general<'a>( .map(|value| value.and_then(|value| value.as_str().map(|value| value.as_bytes()))) .collect_vec(), ))), - DataType::Timestamp(TimeUnit::Microsecond, None) => { - Ok(Arc::new(TimestampMicrosecondArray::from( + DataType::Timestamp(TimeUnit::Microsecond, tz) => match tz { + None => Ok(Arc::new(TimestampMicrosecondArray::from( values .map(|value| { value.and_then(|value| { @@ -749,13 +746,32 @@ fn json_value_to_array_general<'a>( }) }) .collect_vec(), - ))) - } + ))), + Some(tz_str) if tz_str.as_ref() == "UTC" => Ok(Arc::new( + TimestampMicrosecondArray::from( + values + .map(|value| { + value.and_then(|value| { + value.as_str().and_then(TimestampMicrosecondType::parse) + }) + }) + .collect_vec(), + ) + .with_timezone("UTC"), + )), + _ => Err(DeltaTableError::Generic(format!( + "Invalid datatype {}", + datatype + ))), + }, DataType::Date32 => Ok(Arc::new(Date32Array::from( values .map(|value| value.and_then(|value| value.as_str().and_then(Date32Type::parse))) .collect_vec(), ))), - _ => Err(DeltaTableError::Generic("Invalid datatype".to_string())), + _ => Err(DeltaTableError::Generic(format!( + "Invalid datatype {}", + datatype + ))), } } diff --git a/crates/deltalake-core/src/writer/json.rs b/crates/core/src/writer/json.rs similarity index 75% rename from crates/deltalake-core/src/writer/json.rs rename to crates/core/src/writer/json.rs index 0b970ae6d7..8cc908320e 100644 --- a/crates/deltalake-core/src/writer/json.rs +++ b/crates/core/src/writer/json.rs @@ -6,7 +6,7 @@ use std::sync::Arc; use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use arrow::record_batch::*; use bytes::Bytes; -use log::{info, warn}; +use indexmap::IndexMap; use object_store::path::Path; use object_store::ObjectStore; use parquet::{ @@ -14,18 +14,19 @@ use parquet::{ file::properties::WriterProperties, }; use serde_json::Value; +use tracing::{info, warn}; use uuid::Uuid; use super::stats::create_add; use super::utils::{ arrow_schema_without_partitions, next_data_path, record_batch_from_message, - record_batch_without_partitions, stringified_partition_value, + record_batch_without_partitions, }; -use super::{utils::PartitionPath, DeltaWriter, DeltaWriterError}; +use super::{DeltaWriter, DeltaWriterError, WriteMode}; use crate::errors::DeltaTableError; -use crate::kernel::{Add, StructType}; +use crate::kernel::{Add, PartitionsExt, Scalar, StructType}; +use crate::storage::ObjectStoreRetryExt; use crate::table::builder::DeltaTableBuilder; -use crate::table::DeltaTableMetaData; use crate::writer::utils::ShareableBuffer; use crate::DeltaTable; @@ -46,7 +47,7 @@ pub(crate) struct DataArrowWriter { writer_properties: WriterProperties, buffer: ShareableBuffer, arrow_writer: ArrowWriter, - partition_values: HashMap>, + partition_values: IndexMap, buffered_record_batch_count: usize, } @@ -154,7 +155,7 @@ impl DataArrowWriter { writer_properties.clone(), )?; - let partition_values = HashMap::new(); + let partition_values = IndexMap::new(); let buffered_record_batch_count = 0; Ok(Self { @@ -226,30 +227,6 @@ impl JsonWriter { }) } - /// Retrieves the latest schema from table, compares to the current and updates if changed. - /// When schema is updated then `true` is returned which signals the caller that parquet - /// created file or arrow batch should be revisited. - pub fn update_schema( - &mut self, - metadata: &DeltaTableMetaData, - ) -> Result { - let schema: ArrowSchema = - >::try_from(&metadata.schema)?; - - let schema_updated = self.arrow_schema_ref.as_ref() != &schema - || self.partition_columns != metadata.partition_columns; - - if schema_updated { - let _ = std::mem::replace(&mut self.arrow_schema_ref, Arc::new(schema)); - let _ = std::mem::replace( - &mut self.partition_columns, - metadata.partition_columns.clone(), - ); - } - - Ok(schema_updated) - } - /// Returns the current byte length of the in memory buffer. /// This may be used by the caller to decide when to finalize the file write. pub fn buffer_len(&self) -> usize { @@ -310,8 +287,20 @@ impl JsonWriter { #[async_trait::async_trait] impl DeltaWriter> for JsonWriter { - /// Writes the given values to internal parquet buffers for each represented partition. + /// Write a chunk of values into the internal write buffers with the default write mode async fn write(&mut self, values: Vec) -> Result<(), DeltaTableError> { + self.write_with_mode(values, WriteMode::Default).await + } + + /// Writes the given values to internal parquet buffers for each represented partition. + async fn write_with_mode( + &mut self, + values: Vec, + mode: WriteMode, + ) -> Result<(), DeltaTableError> { + if mode != WriteMode::Default { + warn!("The JsonWriter does not currently support non-default write modes, falling back to default mode"); + } let mut partial_writes: Vec<(Value, ParquetError)> = Vec::new(); let arrow_schema = self.arrow_schema(); let divided = self.divide_by_partition_values(values)?; @@ -365,15 +354,14 @@ impl DeltaWriter> for JsonWriter { for (_, writer) in writers { let metadata = writer.arrow_writer.close()?; - let prefix = - PartitionPath::from_hashmap(&self.partition_columns, &writer.partition_values)?; + let prefix = writer.partition_values.hive_partition_path(); let prefix = Path::parse(prefix)?; let uuid = Uuid::new_v4(); let path = next_data_path(&prefix, 0, &uuid, &writer.writer_properties); let obj_bytes = Bytes::from(writer.buffer.to_vec()); let file_size = obj_bytes.len() as i64; - self.storage.put(&path, obj_bytes).await?; + self.storage.put_with_retries(&path, obj_bytes, 15).await?; actions.push(create_add( &writer.partition_values, @@ -423,18 +411,17 @@ fn quarantine_failed_parquet_rows( fn extract_partition_values( partition_cols: &[String], record_batch: &RecordBatch, -) -> Result>, DeltaWriterError> { - let mut partition_values = HashMap::new(); +) -> Result, DeltaWriterError> { + let mut partition_values = IndexMap::new(); for col_name in partition_cols.iter() { let arrow_schema = record_batch.schema(); - let i = arrow_schema.index_of(col_name)?; let col = record_batch.column(i); + let value = Scalar::from_array(col.as_ref(), 0) + .ok_or(DeltaWriterError::MissingPartitionColumn(col_name.clone()))?; - let partition_string = stringified_partition_value(col)?; - - partition_values.insert(col_name.clone(), partition_string); + partition_values.insert(col_name.clone(), value); } Ok(partition_values) @@ -442,6 +429,7 @@ fn extract_partition_values( #[cfg(test)] mod tests { + use arrow_schema::ArrowError; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use std::fs::File; @@ -452,6 +440,7 @@ mod tests { use crate::arrow::datatypes::{ DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, }; + use crate::kernel::DataType; use crate::writer::test_utils::get_delta_schema; use crate::writer::DeltaWriter; use crate::writer::JsonWriter; @@ -524,16 +513,147 @@ mod tests { &record_batch ) .unwrap(), - HashMap::from([ - (String::from("col1"), Some(String::from("1"))), - (String::from("col2"), Some(String::from("2"))), - (String::from("col3"), None), + IndexMap::from([ + (String::from("col1"), Scalar::Integer(1)), + (String::from("col2"), Scalar::Integer(2)), + (String::from("col3"), Scalar::Null(DataType::INTEGER)), ]) ); assert_eq!( extract_partition_values(&[String::from("col1")], &record_batch).unwrap(), - HashMap::from([(String::from("col1"), Some(String::from("1"))),]) + IndexMap::from([(String::from("col1"), Scalar::Integer(1)),]) ); assert!(extract_partition_values(&[String::from("col4")], &record_batch).is_err()) } + + #[tokio::test] + async fn test_parsing_error() { + let table_dir = tempfile::tempdir().unwrap(); + let schema = get_delta_schema(); + let path = table_dir.path().to_str().unwrap().to_string(); + + let arrow_schema = >::try_from(&schema).unwrap(); + let mut writer = JsonWriter::try_new( + path.clone(), + Arc::new(arrow_schema), + Some(vec!["modified".to_string()]), + None, + ) + .unwrap(); + + let data = serde_json::json!( + { + "id" : "A", + "value": "abc", + "modified": "2021-02-01" + } + ); + + let res = writer.write(vec![data]).await; + assert!(matches!( + res, + Err(DeltaTableError::Arrow { + source: ArrowError::JsonError(_) + }) + )); + } + + // The following sets of tests are related to #1386 and mergeSchema support + // + mod schema_evolution { + use super::*; + + #[tokio::test] + async fn test_json_write_mismatched_values() { + let table_dir = tempfile::tempdir().unwrap(); + let schema = get_delta_schema(); + let path = table_dir.path().to_str().unwrap().to_string(); + + let arrow_schema = >::try_from(&schema).unwrap(); + let mut writer = JsonWriter::try_new( + path.clone(), + Arc::new(arrow_schema), + Some(vec!["modified".to_string()]), + None, + ) + .unwrap(); + + let data = serde_json::json!( + { + "id" : "A", + "value": 42, + "modified": "2021-02-01" + } + ); + + writer.write(vec![data]).await.unwrap(); + let add_actions = writer.flush().await.unwrap(); + assert_eq!(add_actions.len(), 1); + + let second_data = serde_json::json!( + { + "id" : 1, + "name" : "Ion" + } + ); + + match writer.write(vec![second_data]).await { + Ok(_) => { + assert!(false, "Should not have successfully written"); + } + _ => {} + } + } + + #[tokio::test] + async fn test_json_write_mismatched_schema() { + use crate::operations::create::CreateBuilder; + let table_dir = tempfile::tempdir().unwrap(); + let schema = get_delta_schema(); + let path = table_dir.path().to_str().unwrap().to_string(); + + let mut table = CreateBuilder::new() + .with_location(&path) + .with_table_name("test-table") + .with_comment("A table for running tests") + .with_columns(schema.fields().clone()) + .await + .unwrap(); + table.load().await.expect("Failed to load table"); + assert_eq!(table.version(), 0); + + let arrow_schema = >::try_from(&schema).unwrap(); + let mut writer = JsonWriter::try_new( + path.clone(), + Arc::new(arrow_schema), + Some(vec!["modified".to_string()]), + None, + ) + .unwrap(); + + let data = serde_json::json!( + { + "id" : "A", + "value": 42, + "modified": "2021-02-01" + } + ); + + writer.write(vec![data]).await.unwrap(); + let add_actions = writer.flush().await.unwrap(); + assert_eq!(add_actions.len(), 1); + + let second_data = serde_json::json!( + { + "postcode" : 1, + "name" : "Ion" + } + ); + + // TODO This should fail because we haven't asked to evolve the schema + writer.write(vec![second_data]).await.unwrap(); + writer.flush_and_commit(&mut table).await.unwrap(); + assert_eq!(table.version(), 1); + } + } } diff --git a/crates/deltalake-core/src/writer/mod.rs b/crates/core/src/writer/mod.rs similarity index 73% rename from crates/deltalake-core/src/writer/mod.rs rename to crates/core/src/writer/mod.rs index fd3d2ed4e7..a392650d24 100644 --- a/crates/deltalake-core/src/writer/mod.rs +++ b/crates/core/src/writer/mod.rs @@ -1,4 +1,3 @@ -#![cfg(all(feature = "arrow", feature = "parquet"))] //! Abstractions and implementations for writing data to delta tables use arrow::{datatypes::SchemaRef, error::ArrowError}; @@ -9,7 +8,7 @@ use serde_json::Value; use crate::errors::DeltaTableError; use crate::kernel::{Action, Add}; -use crate::operations::transaction::commit; +use crate::operations::transaction::CommitBuilder; use crate::protocol::{ColumnCountStat, DeltaOperation, SaveMode}; use crate::DeltaTable; @@ -116,17 +115,35 @@ impl From for DeltaTableError { DeltaWriterError::Io { source } => DeltaTableError::Io { source }, DeltaWriterError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, DeltaWriterError::Parquet { source } => DeltaTableError::Parquet { source }, + DeltaWriterError::DeltaTable(e) => e, + DeltaWriterError::SchemaMismatch { .. } => DeltaTableError::SchemaMismatch { + msg: err.to_string(), + }, _ => DeltaTableError::Generic(err.to_string()), } } } +/// Write mode for the [DeltaWriter] +#[derive(Clone, Debug, PartialEq)] +pub enum WriteMode { + /// Default write mode which will return an error if schemas do not match correctly + Default, + /// Merge the schema of the table with the newly written data + /// + /// [Read more here](https://delta.io/blog/2023-02-08-delta-lake-schema-evolution/) + MergeSchema, +} + #[async_trait] /// Trait for writing data to Delta tables pub trait DeltaWriter { - /// write a chunk of values into the internal write buffers. + /// Write a chunk of values into the internal write buffers with the default write mode async fn write(&mut self, values: T) -> Result<(), DeltaTableError>; + /// Wreite a chunk of values into the internal write buffers with the specified [WriteMode] + async fn write_with_mode(&mut self, values: T, mode: WriteMode) -> Result<(), DeltaTableError>; + /// Flush the internal write buffers to files in the delta table folder structure. /// The corresponding delta [`Add`] actions are returned and should be committed via a transaction. async fn flush(&mut self) -> Result, DeltaTableError>; @@ -135,26 +152,33 @@ pub trait DeltaWriter { /// and commit the changes to the Delta log, creating a new table version. async fn flush_and_commit(&mut self, table: &mut DeltaTable) -> Result { let adds: Vec<_> = self.flush().await?.drain(..).map(Action::Add).collect(); - let partition_cols = table.metadata()?.partition_columns.clone(); - let partition_by = if !partition_cols.is_empty() { - Some(partition_cols) - } else { - None - }; - let operation = DeltaOperation::Write { - mode: SaveMode::Append, - partition_by, - predicate: None, - }; - let version = commit( - table.log_store.as_ref(), - &adds, - operation, - &table.state, - None, - ) - .await?; - table.update().await?; - Ok(version) + flush_and_commit(adds, table).await } } + +/// Method for flushing to be used by writers +pub(crate) async fn flush_and_commit( + adds: Vec, + table: &mut DeltaTable, +) -> Result { + let snapshot = table.snapshot()?; + let partition_cols = snapshot.metadata().partition_columns.clone(); + let partition_by = if !partition_cols.is_empty() { + Some(partition_cols) + } else { + None + }; + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by, + predicate: None, + }; + + let version = CommitBuilder::default() + .with_actions(adds) + .build(Some(snapshot), table.log_store.clone(), operation)? + .await? + .version(); + table.update().await?; + Ok(version) +} diff --git a/crates/deltalake-core/src/writer/record_batch.rs b/crates/core/src/writer/record_batch.rs similarity index 52% rename from crates/deltalake-core/src/writer/record_batch.rs rename to crates/core/src/writer/record_batch.rs index 07240d0335..5c8fb57509 100644 --- a/crates/deltalake-core/src/writer/record_batch.rs +++ b/crates/core/src/writer/record_batch.rs @@ -7,35 +7,40 @@ use std::{collections::HashMap, sync::Arc}; -use arrow::array::{Array, UInt32Array}; +use arrow::array::{new_null_array, Array, UInt32Array}; use arrow::compute::{partition, take}; use arrow::record_batch::RecordBatch; use arrow_array::ArrayRef; use arrow_row::{RowConverter, SortField}; use arrow_schema::{ArrowError, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; +use indexmap::IndexMap; use object_store::{path::Path, ObjectStore}; use parquet::{arrow::ArrowWriter, errors::ParquetError}; use parquet::{basic::Compression, file::properties::WriterProperties}; +use tracing::log::*; use uuid::Uuid; use super::stats::create_add; use super::utils::{ arrow_schema_without_partitions, next_data_path, record_batch_without_partitions, - stringified_partition_value, PartitionPath, ShareableBuffer, + ShareableBuffer, }; -use super::{DeltaWriter, DeltaWriterError}; +use super::{DeltaWriter, DeltaWriterError, WriteMode}; use crate::errors::DeltaTableError; -use crate::kernel::{Add, StructType}; +use crate::kernel::{Action, Add, PartitionsExt, Scalar, StructType}; +use crate::operations::cast::merge_schema; +use crate::storage::ObjectStoreRetryExt; use crate::table::builder::DeltaTableBuilder; -use crate::table::DeltaTableMetaData; use crate::DeltaTable; /// Writes messages to a delta lake table. pub struct RecordBatchWriter { storage: Arc, - arrow_schema_ref: Arc, + arrow_schema_ref: ArrowSchemaRef, + original_schema_ref: ArrowSchemaRef, writer_properties: WriterProperties, + should_evolve: bool, partition_columns: Vec, arrow_writers: HashMap, } @@ -67,9 +72,11 @@ impl RecordBatchWriter { Ok(Self { storage, - arrow_schema_ref: schema, + arrow_schema_ref: schema.clone(), + original_schema_ref: schema, writer_properties, partition_columns: partition_columns.unwrap_or_default(), + should_evolve: false, arrow_writers: HashMap::new(), }) } @@ -91,38 +98,15 @@ impl RecordBatchWriter { Ok(Self { storage: table.object_store(), - arrow_schema_ref, + arrow_schema_ref: arrow_schema_ref.clone(), + original_schema_ref: arrow_schema_ref.clone(), writer_properties, partition_columns, + should_evolve: false, arrow_writers: HashMap::new(), }) } - /// Retrieves the latest schema from table, compares to the current and updates if changed. - /// When schema is updated then `true` is returned which signals the caller that parquet - /// created file or arrow batch should be revisited. - // TODO Test schema update scenarios - pub fn update_schema( - &mut self, - metadata: &DeltaTableMetaData, - ) -> Result { - let schema: ArrowSchema = - >::try_from(&metadata.schema)?; - - let schema_updated = self.arrow_schema_ref.as_ref() != &schema - || self.partition_columns != metadata.partition_columns; - - if schema_updated { - let _ = std::mem::replace(&mut self.arrow_schema_ref, Arc::new(schema)); - let _ = std::mem::replace( - &mut self.partition_columns, - metadata.partition_columns.clone(), - ); - } - - Ok(schema_updated) - } - /// Returns the current byte length of the in memory buffer. /// This may be used by the caller to decide when to finalize the file write. pub fn buffer_len(&self) -> usize { @@ -152,31 +136,29 @@ impl RecordBatchWriter { pub async fn write_partition( &mut self, record_batch: RecordBatch, - partition_values: &HashMap>, - ) -> Result<(), DeltaTableError> { + partition_values: &IndexMap, + mode: WriteMode, + ) -> Result { let arrow_schema = arrow_schema_without_partitions(&self.arrow_schema_ref, &self.partition_columns); - let partition_key = - PartitionPath::from_hashmap(&self.partition_columns, partition_values)?.into(); + let partition_key = partition_values.hive_partition_path(); let record_batch = record_batch_without_partitions(&record_batch, &self.partition_columns)?; - match self.arrow_writers.get_mut(&partition_key) { - Some(writer) => { - writer.write(&record_batch)?; - } + let written_schema = match self.arrow_writers.get_mut(&partition_key) { + Some(writer) => writer.write(&record_batch, mode)?, None => { let mut writer = PartitionWriter::new( arrow_schema, partition_values.clone(), self.writer_properties.clone(), )?; - writer.write(&record_batch)?; + let schema = writer.write(&record_batch, mode)?; let _ = self.arrow_writers.insert(partition_key, writer); + schema } - } - - Ok(()) + }; + Ok(written_schema) } /// Sets the writer properties for the underlying arrow writer. @@ -199,12 +181,26 @@ impl RecordBatchWriter { #[async_trait::async_trait] impl DeltaWriter for RecordBatchWriter { + /// Write a chunk of values into the internal write buffers with the default write mode + async fn write(&mut self, values: RecordBatch) -> Result<(), DeltaTableError> { + self.write_with_mode(values, WriteMode::Default).await + } /// Divides a single record batch into into multiple according to table partitioning. /// Values are written to arrow buffers, to collect data until it should be written to disk. - async fn write(&mut self, values: RecordBatch) -> Result<(), DeltaTableError> { + async fn write_with_mode( + &mut self, + values: RecordBatch, + mode: WriteMode, + ) -> Result<(), DeltaTableError> { + // Set the should_evolve flag for later in case the writer should perform schema evolution + // on its flush_and_commit + self.should_evolve = mode == WriteMode::MergeSchema; + for result in self.divide_by_partition_values(&values)? { - self.write_partition(result.record_batch, &result.partition_values) + let schema = self + .write_partition(result.record_batch, &result.partition_values, mode.clone()) .await?; + self.arrow_schema_ref = schema; } Ok(()) } @@ -216,14 +212,12 @@ impl DeltaWriter for RecordBatchWriter { for (_, writer) in writers { let metadata = writer.arrow_writer.close()?; - let prefix = - PartitionPath::from_hashmap(&self.partition_columns, &writer.partition_values)?; - let prefix = Path::parse(prefix)?; + let prefix = Path::parse(writer.partition_values.hive_partition_path())?; let uuid = Uuid::new_v4(); let path = next_data_path(&prefix, 0, &uuid, &writer.writer_properties); let obj_bytes = Bytes::from(writer.buffer.to_vec()); let file_size = obj_bytes.len() as i64; - self.storage.put(&path, obj_bytes).await?; + self.storage.put_with_retries(&path, obj_bytes, 15).await?; actions.push(create_add( &writer.partition_values, @@ -234,30 +228,47 @@ impl DeltaWriter for RecordBatchWriter { } Ok(actions) } + + /// Flush the internal write buffers to files in the delta table folder structure. + /// and commit the changes to the Delta log, creating a new table version. + async fn flush_and_commit(&mut self, table: &mut DeltaTable) -> Result { + use crate::kernel::{Metadata, StructType}; + let mut adds: Vec = self.flush().await?.drain(..).map(Action::Add).collect(); + + if self.arrow_schema_ref != self.original_schema_ref && self.should_evolve { + let schema: StructType = self.arrow_schema_ref.clone().try_into()?; + // TODO: Handle partition columns somehow? Can we even evolve partition columns? Maybe + // this should just propagate the existing columns in the new action + let part_cols: Vec = vec![]; + let metadata = Metadata::try_new(schema, part_cols, HashMap::new())?; + adds.push(Action::Metadata(metadata)); + } + super::flush_and_commit(adds, table).await + } } /// Helper container for partitioned record batches #[derive(Clone, Debug)] pub struct PartitionResult { /// values found in partition columns - pub partition_values: HashMap>, + pub partition_values: IndexMap, /// remaining dataset with partition column values removed pub record_batch: RecordBatch, } struct PartitionWriter { - arrow_schema: Arc, + arrow_schema: ArrowSchemaRef, writer_properties: WriterProperties, pub(super) buffer: ShareableBuffer, pub(super) arrow_writer: ArrowWriter, - pub(super) partition_values: HashMap>, + pub(super) partition_values: IndexMap, pub(super) buffered_record_batch_count: usize, } impl PartitionWriter { pub fn new( - arrow_schema: Arc, - partition_values: HashMap>, + arrow_schema: ArrowSchemaRef, + partition_values: IndexMap, writer_properties: WriterProperties, ) -> Result { let buffer = ShareableBuffer::default(); @@ -282,21 +293,57 @@ impl PartitionWriter { /// Writes the record batch in-memory and updates internal state accordingly. /// This method buffers the write stream internally so it can be invoked for many /// record batches and flushed after the appropriate number of bytes has been written. - pub fn write(&mut self, record_batch: &RecordBatch) -> Result<(), DeltaWriterError> { - if record_batch.schema() != self.arrow_schema { - return Err(DeltaWriterError::SchemaMismatch { - record_batch_schema: record_batch.schema(), - expected_schema: self.arrow_schema.clone(), - }); - } + /// + /// Returns the schema which was written by the write which can be used to understand if a + /// schema evolution has happened + pub fn write( + &mut self, + record_batch: &RecordBatch, + mode: WriteMode, + ) -> Result { + let merged_batch = if record_batch.schema() != self.arrow_schema { + match mode { + WriteMode::MergeSchema => { + debug!("The writer and record batch schemas do not match, merging"); + + let merged = merge_schema( + self.arrow_schema.as_ref().clone(), + record_batch.schema().as_ref().clone(), + )?; + self.arrow_schema = Arc::new(merged); + + let mut cols = vec![]; + for field in self.arrow_schema.fields() { + if let Some(column) = record_batch.column_by_name(field.name()) { + cols.push(column.clone()); + } else { + let null_column = + new_null_array(field.data_type(), record_batch.num_rows()); + cols.push(null_column); + } + } + Some(RecordBatch::try_new(self.arrow_schema.clone(), cols)?) + } + WriteMode::Default => { + // If the schemas didn't match then an error should be pushed up + Err(DeltaWriterError::SchemaMismatch { + record_batch_schema: record_batch.schema(), + expected_schema: self.arrow_schema.clone(), + })? + } + } + } else { + None + }; // Copy current cursor bytes so we can recover from failures let buffer_bytes = self.buffer.to_vec(); + let record_batch = merged_batch.as_ref().unwrap_or(record_batch); match self.arrow_writer.write(record_batch) { Ok(_) => { self.buffered_record_batch_count += 1; - Ok(()) + Ok(self.arrow_schema.clone()) } // If a write fails we need to reset the state of the PartitionWriter Err(e) => { @@ -330,7 +377,7 @@ pub(crate) fn divide_by_partition_values( if partition_columns.is_empty() { partitions.push(PartitionResult { - partition_values: HashMap::new(), + partition_values: IndexMap::new(), record_batch: values.clone(), }); return Ok(partitions); @@ -358,15 +405,20 @@ pub(crate) fn divide_by_partition_values( .map(|i| Some(indices.value(i))) .collect(); - let partition_key_iter = sorted_partition_columns.iter().map(|col| { - stringified_partition_value(&col.slice(range.start, range.end - range.start)) - }); - - let mut partition_values = HashMap::new(); - for (key, value) in partition_columns.clone().iter().zip(partition_key_iter) { - partition_values.insert(key.clone(), value?); - } + let partition_key_iter = sorted_partition_columns + .iter() + .map(|col| { + Scalar::from_array(&col.slice(range.start, range.end - range.start), 0).ok_or( + DeltaWriterError::MissingPartitionColumn("failed to parse".into()), + ) + }) + .collect::, _>>()?; + let partition_values = partition_columns + .clone() + .into_iter() + .zip(partition_key_iter) + .collect(); let batch_data = arrow_schema .fields() .iter() @@ -398,11 +450,11 @@ fn lexsort_to_indices(arrays: &[ArrayRef]) -> UInt32Array { #[cfg(test)] mod tests { use super::*; - use crate::writer::{ - test_utils::{create_initialized_table, get_record_batch}, - utils::PartitionPath, - }; + use crate::operations::create::CreateBuilder; + use crate::writer::test_utils::*; use arrow::json::ReaderBuilder; + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use std::path::Path; #[tokio::test] @@ -443,7 +495,7 @@ mod tests { String::from("modified=2021-02-01"), String::from("modified=2021-02-02"), ]; - validate_partition_map(partitions, &partition_cols, expected_keys) + validate_partition_map(partitions, expected_keys) } /* @@ -487,8 +539,7 @@ mod tests { "metadata" : {"some-key" : "some-value"}}"# .as_bytes(); - let schema: ArrowSchema = - >::try_from(&delta_schema).unwrap(); + let schema: ArrowSchema = (&delta_schema).try_into().unwrap(); // Using a batch size of two since the buf above only has two records let mut decoder = ReaderBuilder::new(Arc::new(schema)) @@ -511,10 +562,7 @@ mod tests { assert_eq!(partitions.len(), expected_keys.len()); for result in partitions { - let partition_key = - PartitionPath::from_hashmap(&partition_cols, &result.partition_values) - .unwrap() - .into(); + let partition_key = result.partition_values.hive_partition_path(); assert!(expected_keys.contains(&partition_key)); } } @@ -534,7 +582,7 @@ mod tests { String::from("modified=2021-02-02/id=A"), String::from("modified=2021-02-02/id=B"), ]; - validate_partition_map(partitions, &partition_cols.clone(), expected_keys) + validate_partition_map(partitions, expected_keys) } #[tokio::test] @@ -574,20 +622,276 @@ mod tests { } } - fn validate_partition_map( - partitions: Vec, - partition_cols: &[String], - expected_keys: Vec, - ) { + fn validate_partition_map(partitions: Vec, expected_keys: Vec) { assert_eq!(partitions.len(), expected_keys.len()); for result in partitions { - let partition_key = - PartitionPath::from_hashmap(partition_cols, &result.partition_values) - .unwrap() - .into(); + let partition_key = result.partition_values.hive_partition_path(); assert!(expected_keys.contains(&partition_key)); let ref_batch = get_record_batch(Some(partition_key.clone()), false); assert_eq!(ref_batch, result.record_batch); } } + + /// Validates + #[tokio::test] + async fn test_write_tilde() { + use crate::operations::create::CreateBuilder; + let table_schema = crate::writer::test_utils::get_delta_schema(); + let partition_cols = vec!["modified".to_string(), "id".to_string()]; + let table_dir = tempfile::Builder::new() + .prefix("example~with~tilde") + .tempdir() + .unwrap(); + let table_path = table_dir.path(); + + let table = CreateBuilder::new() + .with_location(table_path.to_str().unwrap()) + .with_table_name("test-table") + .with_comment("A table for running tests") + .with_columns(table_schema.fields().clone()) + .with_partition_columns(partition_cols) + .await + .unwrap(); + + let batch = get_record_batch(None, false); + let mut writer = RecordBatchWriter::for_table(&table).unwrap(); + + writer.write(batch).await.unwrap(); + let adds = writer.flush().await.unwrap(); + assert_eq!(adds.len(), 4); + } + + // The following sets of tests are related to #1386 and mergeSchema support + // + mod schema_evolution { + use super::*; + + #[tokio::test] + async fn test_write_mismatched_schema() { + let batch = get_record_batch(None, false); + let partition_cols = vec![]; + let table = create_initialized_table(&partition_cols).await; + let mut writer = RecordBatchWriter::for_table(&table).unwrap(); + + // Write the first batch with the first schema to the table + writer.write(batch).await.unwrap(); + let adds = writer.flush().await.unwrap(); + assert_eq!(adds.len(), 1); + + // Create a second batch with a different schema + let second_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ])); + let second_batch = RecordBatch::try_new( + second_schema, + vec![ + Arc::new(Int32Array::from(vec![Some(1), Some(2)])), + Arc::new(StringArray::from(vec![Some("will"), Some("robert")])), + ], + ) + .unwrap(); + + let result = writer.write(second_batch).await; + assert!(result.is_err()); + + match result { + Ok(_) => { + assert!(false, "Should not have successfully written"); + } + Err(e) => { + match e { + DeltaTableError::SchemaMismatch { .. } => { + // this is expected + } + others => { + assert!(false, "Got the wrong error: {others:?}"); + } + } + } + }; + } + + #[tokio::test] + async fn test_write_schema_evolution() { + let table_schema = get_delta_schema(); + let table_dir = tempfile::tempdir().unwrap(); + let table_path = table_dir.path(); + + let mut table = CreateBuilder::new() + .with_location(table_path.to_str().unwrap()) + .with_table_name("test-table") + .with_comment("A table for running tests") + .with_columns(table_schema.fields().clone()) + .await + .unwrap(); + table.load().await.expect("Failed to load table"); + assert_eq!(table.version(), 0); + + let batch = get_record_batch(None, false); + let mut writer = RecordBatchWriter::for_table(&table).unwrap(); + + writer.write(batch).await.unwrap(); + let version = writer.flush_and_commit(&mut table).await.unwrap(); + assert_eq!(version, 1); + table.load().await.expect("Failed to load table"); + assert_eq!(table.version(), 1); + + // Create a second batch with a different schema + let second_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("vid", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ])); + let second_batch = RecordBatch::try_new( + second_schema, + vec![ + Arc::new(Int32Array::from(vec![Some(1), Some(2)])), // vid + Arc::new(StringArray::from(vec![Some("will"), Some("robert")])), // name + ], + ) + .unwrap(); + + let result = writer + .write_with_mode(second_batch, WriteMode::MergeSchema) + .await; + assert!( + result.is_ok(), + "Failed to write with WriteMode::MergeSchema, {:?}", + result + ); + let version = writer.flush_and_commit(&mut table).await.unwrap(); + assert_eq!(version, 2); + table.load().await.expect("Failed to load table"); + assert_eq!(table.version(), 2); + + let new_schema = table.metadata().unwrap().schema().unwrap(); + let expected_columns = vec!["id", "value", "modified", "vid", "name"]; + let found_columns: Vec<&String> = + new_schema.fields().iter().map(|f| f.name()).collect(); + assert_eq!( + expected_columns, found_columns, + "The new table schema does not contain all evolved columns as expected" + ); + } + + #[tokio::test] + async fn test_schema_evolution_column_type_mismatch() { + let batch = get_record_batch(None, false); + let partition_cols = vec![]; + let mut table = create_initialized_table(&partition_cols).await; + + let mut writer = RecordBatchWriter::for_table(&table).unwrap(); + + // Write the first batch with the first schema to the table + writer.write(batch).await.unwrap(); + let version = writer.flush_and_commit(&mut table).await.unwrap(); + assert_eq!(version, 1); + + // Create a second batch with a different schema + let second_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ])); + let second_batch = RecordBatch::try_new( + second_schema, + vec![ + Arc::new(Int32Array::from(vec![Some(1), Some(2)])), // vid + Arc::new(StringArray::from(vec![Some("will"), Some("robert")])), // name + ], + ) + .unwrap(); + + let result = writer + .write_with_mode(second_batch, WriteMode::MergeSchema) + .await; + assert!( + result.is_err(), + "Did not expect to successfully add new writes with different column types: {:?}", + result + ); + } + + #[tokio::test] + async fn test_schema_evolution_with_nonnullable_col() { + use crate::kernel::{ + DataType as DeltaDataType, PrimitiveType, StructField, StructType, + }; + + let table_schema = StructType::new(vec![ + StructField::new( + "id".to_string(), + DeltaDataType::Primitive(PrimitiveType::String), + false, + ), + StructField::new( + "value".to_string(), + DeltaDataType::Primitive(PrimitiveType::Integer), + true, + ), + StructField::new( + "modified".to_string(), + DeltaDataType::Primitive(PrimitiveType::String), + true, + ), + ]); + let table_dir = tempfile::tempdir().unwrap(); + let table_path = table_dir.path(); + + let mut table = CreateBuilder::new() + .with_location(table_path.to_str().unwrap()) + .with_table_name("test-table") + .with_comment("A table for running tests") + .with_columns(table_schema.fields().clone()) + .await + .unwrap(); + table.load().await.expect("Failed to load table"); + assert_eq!(table.version(), 0); + + // Hand-crafting the first RecordBatch to ensure that a write with non-nullable columns + // works properly before attepting the second write + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("value", DataType::Int32, true), + Field::new("modified", DataType::Utf8, true), + ])); + let batch = RecordBatch::try_new( + arrow_schema, + vec![ + Arc::new(StringArray::from(vec![Some("1"), Some("2")])), // id + Arc::new(new_null_array(&DataType::Int32, 2)), // value + Arc::new(new_null_array(&DataType::Utf8, 2)), // modified + ], + ) + .unwrap(); + + // Write the first batch with the first schema to the table + let mut writer = RecordBatchWriter::for_table(&table).unwrap(); + writer.write(batch).await.unwrap(); + let version = writer.flush_and_commit(&mut table).await.unwrap(); + assert_eq!(version, 1); + + // Create a second batch with a different schema + let second_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "name", + DataType::Utf8, + true, + )])); + let second_batch = RecordBatch::try_new( + second_schema, + vec![ + Arc::new(StringArray::from(vec![Some("will"), Some("robert")])), // name + ], + ) + .unwrap(); + + let result = writer + .write_with_mode(second_batch, WriteMode::MergeSchema) + .await; + assert!( + result.is_err(), + "Should not have been able to write with a missing non-nullable column: {:?}", + result + ); + } + } } diff --git a/crates/deltalake-core/src/writer/stats.rs b/crates/core/src/writer/stats.rs similarity index 98% rename from crates/deltalake-core/src/writer/stats.rs rename to crates/core/src/writer/stats.rs index cc3badc1fa..20201c949c 100644 --- a/crates/deltalake-core/src/writer/stats.rs +++ b/crates/core/src/writer/stats.rs @@ -2,6 +2,7 @@ use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; use std::{collections::HashMap, ops::AddAssign}; +use indexmap::IndexMap; use parquet::format::FileMetaData; use parquet::schema::types::{ColumnDescriptor, SchemaDescriptor}; use parquet::{basic::LogicalType, errors::ParquetError}; @@ -11,12 +12,12 @@ use parquet::{ }; use super::*; -use crate::kernel::Add; +use crate::kernel::{Add, Scalar}; use crate::protocol::{ColumnValueStat, Stats}; /// Creates an [`Add`] log action struct. pub fn create_add( - partition_values: &HashMap>, + partition_values: &IndexMap, path: String, size: i64, file_metadata: &FileMetaData, @@ -32,7 +33,19 @@ pub fn create_add( Ok(Add { path, size, - partition_values: partition_values.to_owned(), + partition_values: partition_values + .iter() + .map(|(k, v)| { + ( + k.clone(), + if v.is_null() { + None + } else { + Some(v.serialize()) + }, + ) + }) + .collect(), modification_time, data_change: true, stats: Some(stats_string), @@ -41,13 +54,12 @@ pub fn create_add( base_row_id: None, default_row_commit_version: None, stats_parsed: None, - partition_values_parsed: None, clustering_provider: None, }) } fn stats_from_file_metadata( - partition_values: &HashMap>, + partition_values: &IndexMap, file_metadata: &FileMetaData, ) -> Result { let type_ptr = parquet::schema::types::from_thrift(file_metadata.schema.as_slice()); diff --git a/crates/deltalake-core/src/writer/test_utils.rs b/crates/core/src/writer/test_utils.rs similarity index 91% rename from crates/deltalake-core/src/writer/test_utils.rs rename to crates/core/src/writer/test_utils.rs index 1daf9e407b..093ad7cbd0 100644 --- a/crates/deltalake-core/src/writer/test_utils.rs +++ b/crates/core/src/writer/test_utils.rs @@ -7,10 +7,9 @@ use arrow::compute::take; use arrow_array::{Int32Array, Int64Array, RecordBatch, StringArray, StructArray, UInt32Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; -use crate::kernel::{DataType as DeltaDataType, PrimitiveType, StructField, StructType}; +use crate::kernel::{DataType as DeltaDataType, Metadata, PrimitiveType, StructField, StructType}; use crate::operations::create::CreateBuilder; use crate::operations::DeltaOps; -use crate::table::DeltaTableMetaData; use crate::{DeltaConfigKey, DeltaTable, DeltaTableBuilder}; pub type TestResult = Result<(), Box>; @@ -152,16 +151,9 @@ pub fn get_delta_schema() -> StructType { ]) } -pub fn get_delta_metadata(partition_cols: &[String]) -> DeltaTableMetaData { +pub fn get_delta_metadata(partition_cols: &[String]) -> Metadata { let table_schema = get_delta_schema(); - DeltaTableMetaData::new( - None, - None, - None, - table_schema, - partition_cols.to_vec(), - HashMap::new(), - ) + Metadata::try_new(table_schema, partition_cols.to_vec(), HashMap::new()).unwrap() } pub fn get_record_batch_with_nested_struct() -> RecordBatch { @@ -323,7 +315,8 @@ pub mod datafusion { use std::sync::Arc; pub async fn get_data(table: &DeltaTable) -> Vec { - let table = DeltaTable::new_with_state(table.log_store.clone(), table.state.clone()); + let table = + DeltaTable::new_with_state(table.log_store.clone(), table.snapshot().unwrap().clone()); let ctx = SessionContext::new(); ctx.register_table("test", Arc::new(table)).unwrap(); ctx.sql("select * from test") @@ -334,6 +327,24 @@ pub mod datafusion { .unwrap() } + pub async fn get_data_sorted(table: &DeltaTable, columns: &str) -> Vec { + let table = DeltaTable::new_with_state( + table.log_store.clone(), + table.state.as_ref().unwrap().clone(), + ); + let ctx = SessionContext::new(); + ctx.register_table("test", Arc::new(table)).unwrap(); + ctx.sql(&format!( + "select {} from test order by {}", + columns, columns + )) + .await + .unwrap() + .collect() + .await + .unwrap() + } + pub async fn write_batch(table: DeltaTable, batch: RecordBatch) -> DeltaTable { DeltaOps(table) .write(vec![batch.clone()]) diff --git a/crates/core/src/writer/utils.rs b/crates/core/src/writer/utils.rs new file mode 100644 index 0000000000..3c95942993 --- /dev/null +++ b/crates/core/src/writer/utils.rs @@ -0,0 +1,228 @@ +//! Handle JSON messages when writing to delta tables +//! + +use std::io::Write; +use std::sync::Arc; + +use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; +use arrow::json::ReaderBuilder; +use arrow::record_batch::*; +use object_store::path::Path; +use parking_lot::RwLock; +use parquet::basic::Compression; +use parquet::file::properties::WriterProperties; +use parquet::schema::types::ColumnPath; +use serde_json::Value; +use uuid::Uuid; + +use crate::errors::DeltaResult; +use crate::writer::DeltaWriterError; + +/// Generate the name of the file to be written +/// prefix: The location of the file to be written +/// part_count: Used the indicate that single logical partition was split into multiple physical files +/// starts at 0. Is typically used when writer splits that data due to file size constraints +pub(crate) fn next_data_path( + prefix: &Path, + part_count: usize, + writer_id: &Uuid, + writer_properties: &WriterProperties, +) -> Path { + fn compression_to_str(compression: &Compression) -> &str { + match compression { + // This is to match HADOOP's convention + // https://github.com/apache/parquet-mr/blob/c4977579ab3b149ea045a177b039f055b5408e8f/parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java#L27-L34 + Compression::UNCOMPRESSED => "", + Compression::SNAPPY => ".snappy", + Compression::GZIP(_) => ".gz", + Compression::LZO => ".lzo", + Compression::BROTLI(_) => ".br", + Compression::LZ4 => ".lz4", + Compression::ZSTD(_) => ".zstd", + Compression::LZ4_RAW => ".lz4raw", + } + } + + // We can not access the default column properties but the current implementation will return + // the default compression when the column is not found + let column_path = ColumnPath::new(Vec::new()); + let compression = writer_properties.compression(&column_path); + + let part = format!("{:0>5}", part_count); + + // TODO: what does c000 mean? + let file_name = format!( + "part-{}-{}-c000{}.parquet", + part, + writer_id, + compression_to_str(&compression) + ); + prefix.child(file_name) +} + +/// Convert a vector of json values to a RecordBatch +pub fn record_batch_from_message( + arrow_schema: Arc, + json: &[Value], +) -> DeltaResult { + let mut decoder = ReaderBuilder::new(arrow_schema).build_decoder().unwrap(); + decoder.serialize(json)?; + decoder + .flush()? + .ok_or_else(|| DeltaWriterError::EmptyRecordBatch.into()) +} + +/// Remove any partition related columns from the record batch +pub(crate) fn record_batch_without_partitions( + record_batch: &RecordBatch, + partition_columns: &[String], +) -> Result { + let mut non_partition_columns = Vec::new(); + for (i, field) in record_batch.schema().fields().iter().enumerate() { + if !partition_columns.contains(field.name()) { + non_partition_columns.push(i); + } + } + + Ok(record_batch.project(&non_partition_columns)?) +} + +/// Arrow schema for the physical file which has partition columns removed +pub(crate) fn arrow_schema_without_partitions( + arrow_schema: &Arc, + partition_columns: &[String], +) -> ArrowSchemaRef { + Arc::new(ArrowSchema::new( + arrow_schema + .fields() + .iter() + .filter(|f| !partition_columns.contains(f.name())) + .map(|f| f.to_owned()) + .collect::>(), + )) +} + +/// An in memory buffer that allows for shared ownership and interior mutability. +/// The underlying buffer is wrapped in an `Arc` and `RwLock`, so cloning the instance +/// allows multiple owners to have access to the same underlying buffer. +#[derive(Debug, Default, Clone)] +pub struct ShareableBuffer { + buffer: Arc>>, +} + +impl ShareableBuffer { + /// Consumes this instance and returns the underlying buffer. + /// Returns None if there are other references to the instance. + pub fn into_inner(self) -> Option> { + Arc::try_unwrap(self.buffer) + .ok() + .map(|lock| lock.into_inner()) + } + + /// Returns a clone of the the underlying buffer as a `Vec`. + pub fn to_vec(&self) -> Vec { + let inner = self.buffer.read(); + (*inner).to_vec() + } + + /// Returns the number of bytes in the underlying buffer. + pub fn len(&self) -> usize { + let inner = self.buffer.read(); + (*inner).len() + } + + /// Returns true if the underlying buffer is empty. + pub fn is_empty(&self) -> bool { + let inner = self.buffer.read(); + (*inner).is_empty() + } + + /// Creates a new instance with buffer initialized from the underylying bytes. + pub fn from_bytes(bytes: &[u8]) -> Self { + Self { + buffer: Arc::new(RwLock::new(bytes.to_vec())), + } + } +} + +impl Write for ShareableBuffer { + fn write(&mut self, buf: &[u8]) -> std::io::Result { + let mut inner = self.buffer.write(); + (*inner).write(buf) + } + + fn flush(&mut self) -> std::io::Result<()> { + let mut inner = self.buffer.write(); + (*inner).flush() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}; + + #[test] + fn test_data_path() { + let prefix = Path::parse("x=0/y=0").unwrap(); + let uuid = Uuid::parse_str("02f09a3f-1624-3b1d-8409-44eff7708208").unwrap(); + + // Validated against Spark + let props = WriterProperties::builder() + .set_compression(Compression::UNCOMPRESSED) + .build(); + + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.parquet" + ); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.snappy.parquet" + ); + + let props = WriterProperties::builder() + .set_compression(Compression::GZIP(GzipLevel::default())) + .build(); + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.gz.parquet" + ); + + let props = WriterProperties::builder() + .set_compression(Compression::LZ4) + .build(); + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.lz4.parquet" + ); + + let props = WriterProperties::builder() + .set_compression(Compression::ZSTD(ZstdLevel::default())) + .build(); + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.zstd.parquet" + ); + + let props = WriterProperties::builder() + .set_compression(Compression::LZ4_RAW) + .build(); + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.lz4raw.parquet" + ); + + let props = WriterProperties::builder() + .set_compression(Compression::BROTLI(BrotliLevel::default())) + .build(); + assert_eq!( + next_data_path(&prefix, 1, &uuid, &props).as_ref(), + "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.br.parquet" + ); + } +} diff --git a/crates/deltalake-core/tests/checkpoint_writer.rs b/crates/core/tests/checkpoint_writer.rs similarity index 88% rename from crates/deltalake-core/tests/checkpoint_writer.rs rename to crates/core/tests/checkpoint_writer.rs index b1fc28faed..696e379569 100644 --- a/crates/deltalake-core/tests/checkpoint_writer.rs +++ b/crates/core/tests/checkpoint_writer.rs @@ -1,11 +1,9 @@ -#[cfg(all(feature = "arrow", feature = "parquet"))] mod fs_common; use deltalake_core::protocol::DeltaOperation; // NOTE: The below is a useful external command for inspecting the written checkpoint schema visually: // parquet-tools inspect tests/data/checkpoints/_delta_log/00000000000000000005.checkpoint.parquet -#[cfg(all(feature = "arrow", feature = "parquet"))] mod simple_checkpoint { use deltalake_core::*; use pretty_assertions::assert_eq; @@ -14,7 +12,7 @@ mod simple_checkpoint { #[tokio::test] async fn simple_checkpoint_test() { - let table_location = "./tests/data/checkpoints"; + let table_location = "../test/tests/data/checkpoints"; let table_path = PathBuf::from(table_location); let log_path = table_path.join("_delta_log"); @@ -51,8 +49,8 @@ mod simple_checkpoint { // delta table should load just fine with the checkpoint in place let table_result = deltalake_core::open_table(table_location).await.unwrap(); let table = table_result; - let files = table.get_files(); - assert_eq!(12, files.len()); + let files = table.get_files_iter().unwrap(); + assert_eq!(12, files.count()); } fn get_last_checkpoint_version(log_path: &Path) -> i64 { @@ -84,7 +82,6 @@ mod simple_checkpoint { } } -#[cfg(all(feature = "arrow", feature = "parquet"))] mod delete_expired_delta_log_in_checkpoint { use super::*; @@ -97,7 +94,7 @@ mod delete_expired_delta_log_in_checkpoint { #[tokio::test] async fn test_delete_expired_logs() { let mut table = fs_common::create_table( - "./tests/data/checkpoints_with_expired_logs/expired", + "../test/tests/data/checkpoints_with_expired_logs/expired", Some(hashmap! { DeltaConfigKey::LogRetentionDuration.as_ref().into() => Some("interval 10 minute".to_string()), DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into() => Some("true".to_string()) @@ -138,10 +135,10 @@ mod delete_expired_delta_log_in_checkpoint { table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files(), + table.get_files_iter().unwrap().collect::>(), vec![ + ObjectStorePath::from(a2.path.as_ref()), ObjectStorePath::from(a1.path.as_ref()), - ObjectStorePath::from(a2.path.as_ref()) ] ); @@ -161,7 +158,7 @@ mod delete_expired_delta_log_in_checkpoint { #[tokio::test] async fn test_not_delete_expired_logs() { let mut table = fs_common::create_table( - "./tests/data/checkpoints_with_expired_logs/not_delete_expired", + "../test/tests/data/checkpoints_with_expired_logs/not_delete_expired", Some(hashmap! { DeltaConfigKey::LogRetentionDuration.as_ref().into() => Some("interval 1 second".to_string()), DeltaConfigKey::EnableExpiredLogCleanup.as_ref().into() => Some("false".to_string()) @@ -186,10 +183,10 @@ mod delete_expired_delta_log_in_checkpoint { .unwrap(); table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files(), + table.get_files_iter().unwrap().collect::>(), vec![ + ObjectStorePath::from(a2.path.as_ref()), ObjectStorePath::from(a1.path.as_ref()), - ObjectStorePath::from(a2.path.as_ref()) ] ); @@ -206,7 +203,6 @@ mod delete_expired_delta_log_in_checkpoint { } } -#[cfg(all(feature = "arrow", feature = "parquet"))] mod checkpoints_with_tombstones { use super::*; use ::object_store::path::Path as ObjectStorePath; @@ -218,7 +214,7 @@ mod checkpoints_with_tombstones { use parquet::file::reader::{FileReader, SerializedFileReader}; use parquet::schema::types::Type; use pretty_assertions::assert_eq; - use std::collections::HashSet; + use std::collections::{HashMap, HashSet}; use std::fs::File; use std::iter::FromIterator; use uuid::Uuid; @@ -236,8 +232,9 @@ mod checkpoints_with_tombstones { } #[tokio::test] + #[ignore] async fn test_expired_tombstones() { - let mut table = fs_common::create_table("./tests/data/checkpoints_tombstones/expired", Some(hashmap! { + let mut table = fs_common::create_table("../test/tests/data/checkpoints_tombstones/expired", Some(hashmap! { DeltaConfigKey::DeletedFileRetentionDuration.as_ref().into() => Some("interval 1 minute".to_string()) })).await; @@ -249,32 +246,52 @@ mod checkpoints_with_tombstones { checkpoints::create_checkpoint(&table).await.unwrap(); table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files(), + table.get_files_iter().unwrap().collect::>(), vec![ + ObjectStorePath::from(a2.path.as_ref()), ObjectStorePath::from(a1.path.as_ref()), - ObjectStorePath::from(a2.path.as_ref()) ] ); let (removes1, opt1) = pseudo_optimize(&mut table, 5 * 59 * 1000).await; assert_eq!( - table.get_files(), + table.get_files_iter().unwrap().collect::>(), vec![ObjectStorePath::from(opt1.path.as_ref())] ); - assert_eq!(table.get_state().all_tombstones(), &removes1); + + assert_eq!( + table + .snapshot() + .unwrap() + .all_tombstones(table.object_store().clone()) + .await + .unwrap() + .collect::>(), + removes1 + ); checkpoints::create_checkpoint(&table).await.unwrap(); table.update().await.unwrap(); // make table to read the checkpoint assert_eq!( - table.get_files(), + table.get_files_iter().unwrap().collect::>(), vec![ObjectStorePath::from(opt1.path.as_ref())] ); - assert_eq!(table.get_state().all_tombstones().len(), 0); // stale removes are deleted from the state + assert_eq!( + table + .snapshot() + .unwrap() + .all_tombstones(table.object_store().clone()) + .await + .unwrap() + .count(), + 0 + ); // stale removes are deleted from the state } #[tokio::test] + #[ignore] async fn test_checkpoint_with_extended_file_metadata_true() { - let path = "./tests/data/checkpoints_tombstones/metadata_true"; + let path = "../test/tests/data/checkpoints_tombstones/metadata_true"; let mut table = fs_common::create_table(path, None).await; let r1 = remove_metadata_true(); let r2 = remove_metadata_true(); @@ -290,7 +307,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_checkpoint_with_extended_file_metadata_false() { - let path = "./tests/data/checkpoints_tombstones/metadata_false"; + let path = "../test/tests/data/checkpoints_tombstones/metadata_false"; let mut table = fs_common::create_table(path, None).await; let r1 = remove_metadata_true(); let r2 = remove_metadata_false(); @@ -313,7 +330,7 @@ mod checkpoints_with_tombstones { #[tokio::test] async fn test_checkpoint_with_extended_file_metadata_broken() { - let path = "./tests/data/checkpoints_tombstones/metadata_broken"; + let path = "../test/tests/data/checkpoints_tombstones/metadata_broken"; let mut table = fs_common::create_table(path, None).await; let r1 = remove_metadata_broken(); let r2 = remove_metadata_false(); @@ -335,16 +352,16 @@ mod checkpoints_with_tombstones { async fn pseudo_optimize(table: &mut DeltaTable, offset_millis: i64) -> (HashSet, Add) { let removes: HashSet = table - .get_files() - .iter() + .get_files_iter() + .unwrap() .map(|p| Remove { path: p.to_string(), deletion_timestamp: Some(Utc::now().timestamp_millis() - offset_millis), data_change: false, extended_file_metadata: None, - partition_values: None, + partition_values: Some(HashMap::new()), size: None, - tags: None, + tags: Some(HashMap::new()), deletion_vector: None, base_row_id: None, default_row_commit_version: None, diff --git a/crates/deltalake-core/tests/command_filesystem_check.rs b/crates/core/tests/command_filesystem_check.rs similarity index 52% rename from crates/deltalake-core/tests/command_filesystem_check.rs rename to crates/core/tests/command_filesystem_check.rs index ac11c5d376..fdc1e6fae7 100644 --- a/crates/deltalake-core/tests/command_filesystem_check.rs +++ b/crates/core/tests/command_filesystem_check.rs @@ -1,52 +1,19 @@ -#![cfg(feature = "integration_test")] +use std::collections::HashSet; -use deltalake_core::test_utils::{ - set_env_if_not_set, IntegrationContext, StorageIntegration, TestResult, TestTables, -}; use deltalake_core::Path; use deltalake_core::{errors::DeltaTableError, DeltaOps}; +use deltalake_test::utils::*; use serial_test::serial; -mod common; - #[tokio::test] #[serial] async fn test_filesystem_check_local() -> TestResult { - test_filesystem_check(StorageIntegration::Local).await -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[tokio::test] -#[serial] -async fn test_filesystem_check_aws() -> TestResult { - set_env_if_not_set("AWS_S3_ALLOW_UNSAFE_RENAME", "true"); - set_env_if_not_set("AWS_S3_LOCKING_PROVIDER", "none"); - test_filesystem_check(StorageIntegration::Amazon).await -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_filesystem_check_azure() -> TestResult { - test_filesystem_check(StorageIntegration::Microsoft).await -} - -#[cfg(feature = "gcs")] -#[tokio::test] -#[serial] -async fn test_filesystem_check_gcp() -> TestResult { - test_filesystem_check(StorageIntegration::Google).await -} - -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn test_filesystem_check_hdfs() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Hdfs).await?) + let storage = Box::::default(); + let context = IntegrationContext::new(storage)?; + test_filesystem_check(&context).await } -async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(storage)?; +async fn test_filesystem_check(context: &IntegrationContext) -> TestResult { context.load_table(TestTables::Simple).await?; let file = "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"; let path = Path::from_iter([&TestTables::Simple.as_name(), file]); @@ -55,31 +22,36 @@ async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { context.object_store().delete(&path).await?; let table = context.table_builder(TestTables::Simple).load().await?; - let version = table.state.version(); - let active = table.state.files().len(); + let version = table.snapshot()?.version(); + let active = table.snapshot()?.files_count(); // Validate a Dry run does not mutate the table log and indentifies orphaned add actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().with_dry_run(true).await?; - assert_eq!(version, table.state.version()); - assert_eq!(active, table.state.files().len()); + assert_eq!(version, table.snapshot()?.version()); + assert_eq!(active, table.snapshot()?.files_count()); assert_eq!(vec![file.to_string()], metrics.files_removed); // Validate a run updates the table version with proper remove actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().await?; - assert_eq!(version + 1, table.state.version()); - assert_eq!(active - 1, table.state.files().len()); + assert_eq!(version + 1, table.snapshot()?.version()); + assert_eq!(active - 1, table.snapshot()?.files_count()); assert_eq!(vec![file.to_string()], metrics.files_removed); - let remove = table.state.all_tombstones().get(file).unwrap(); + let remove = table + .snapshot()? + .all_tombstones(table.object_store().clone()) + .await? + .collect::>(); + let remove = remove.get(file).unwrap(); assert!(remove.data_change); // An additional run should return an empty list of orphaned actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().await?; - assert_eq!(version + 1, table.state.version()); - assert_eq!(active - 1, table.state.files().len()); + assert_eq!(version + 1, table.snapshot()?.version()); + assert_eq!(active - 1, table.snapshot()?.files_count()); assert!(metrics.files_removed.is_empty()); Ok(()) @@ -88,7 +60,7 @@ async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { #[tokio::test] #[serial] async fn test_filesystem_check_partitioned() -> TestResult { - let storage = StorageIntegration::Local; + let storage = Box::::default(); let context = IntegrationContext::new(storage)?; context .load_table(TestTables::Delta0_8_0Partitioned) @@ -103,17 +75,23 @@ async fn test_filesystem_check_partitioned() -> TestResult { .table_builder(TestTables::Delta0_8_0Partitioned) .load() .await?; - let version = table.state.version(); - let active = table.state.files().len(); + + let version = table.snapshot()?.version(); + let active = table.snapshot()?.files_count(); // Validate a run updates the table version with proper remove actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().await?; - assert_eq!(version + 1, table.state.version()); - assert_eq!(active - 1, table.state.files().len()); + assert_eq!(version + 1, table.snapshot()?.version()); + assert_eq!(active - 1, table.snapshot()?.files_count()); assert_eq!(vec![file.to_string()], metrics.files_removed); - let remove = table.state.all_tombstones().get(file).unwrap(); + let remove = table + .snapshot()? + .all_tombstones(table.object_store().clone()) + .await? + .collect::>(); + let remove = remove.get(file).unwrap(); assert!(remove.data_change); Ok(()) } @@ -122,7 +100,8 @@ async fn test_filesystem_check_partitioned() -> TestResult { #[serial] async fn test_filesystem_check_fails_for_concurrent_delete() -> TestResult { // Validate failure when a non dry only executes on the latest version - let context = IntegrationContext::new(StorageIntegration::Local)?; + let storage = Box::::default(); + let context = IntegrationContext::new(storage)?; context.load_table(TestTables::Simple).await?; let file = "part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet"; let path = Path::from_iter([&TestTables::Simple.as_name(), file]); @@ -144,34 +123,3 @@ async fn test_filesystem_check_fails_for_concurrent_delete() -> TestResult { Ok(()) } - -#[tokio::test] -#[serial] -#[ignore = "should this actually fail? with conflict resolution, we are re-trying again."] -async fn test_filesystem_check_outdated() -> TestResult { - // Validate failure when a non dry only executes on the latest version - let context = IntegrationContext::new(StorageIntegration::Local)?; - context.load_table(TestTables::Simple).await?; - let file = "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"; - let path = Path::from_iter([&TestTables::Simple.as_name(), file]); - - // Delete an active file from underlying storage without an update to the log to simulate an external fault - context.object_store().delete(&path).await?; - - let table = context - .table_builder(TestTables::Simple) - .with_version(2) - .load() - .await?; - - let op = DeltaOps::from(table); - let res = op.filesystem_check().with_dry_run(false).await; - println!("{:?}", res); - if let Err(DeltaTableError::VersionAlreadyExists(version)) = res { - assert!(version == 3); - } else { - panic!(); - } - - Ok(()) -} diff --git a/crates/deltalake-core/tests/command_optimize.rs b/crates/core/tests/command_optimize.rs similarity index 91% rename from crates/deltalake-core/tests/command_optimize.rs rename to crates/core/tests/command_optimize.rs index b91558ce08..823ba75304 100644 --- a/crates/deltalake-core/tests/command_optimize.rs +++ b/crates/core/tests/command_optimize.rs @@ -1,17 +1,15 @@ -#![cfg(all(feature = "arrow", feature = "parquet"))] - -use std::time::{Duration, SystemTime, UNIX_EPOCH}; -use std::{collections::HashMap, error::Error, sync::Arc}; +use std::time::Duration; +use std::{error::Error, sync::Arc}; use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType as ArrowDataType, Field, Schema as ArrowSchema}; use arrow_select::concat::concat_batches; use deltalake_core::errors::DeltaTableError; -use deltalake_core::kernel::{Action, DataType, PrimitiveType, Remove, StructField}; +use deltalake_core::kernel::{Action, DataType, PrimitiveType, StructField}; use deltalake_core::operations::optimize::{ create_merge_plan, MetricDetails, Metrics, OptimizeType, }; -use deltalake_core::operations::transaction::commit; +use deltalake_core::operations::transaction::{CommitBuilder, CommitProperties}; use deltalake_core::operations::DeltaOps; use deltalake_core::protocol::DeltaOperation; use deltalake_core::storage::ObjectStoreRef; @@ -24,7 +22,7 @@ use parquet::arrow::ParquetRecordBatchStreamBuilder; use parquet::file::properties::WriterProperties; use rand::prelude::*; use serde_json::json; -use tempdir::TempDir; +use tempfile::TempDir; struct Context { pub tmp_dir: TempDir, @@ -56,7 +54,7 @@ async fn setup_test(partitioned: bool) -> Result> { vec![] }; - let tmp_dir = tempdir::TempDir::new("opt_table").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let table_uri = tmp_dir.path().to_str().to_owned().unwrap(); let dt = DeltaOps::try_from_uri(table_uri) .await? @@ -170,7 +168,7 @@ async fn test_optimize_non_partitioned_table() -> Result<(), Box> { .await?; let version = dt.version(); - assert_eq!(dt.get_state().files().len(), 5); + assert_eq!(dt.get_files_count(), 5); let optimize = DeltaOps(dt).optimize().with_target_size(2_000_000); let (dt, metrics) = optimize.await?; @@ -180,7 +178,13 @@ async fn test_optimize_non_partitioned_table() -> Result<(), Box> { assert_eq!(metrics.num_files_removed, 4); assert_eq!(metrics.total_considered_files, 5); assert_eq!(metrics.partitions_optimized, 1); - assert_eq!(dt.get_state().files().len(), 2); + assert_eq!(dt.get_files_count(), 2); + + let commit_info = dt.history(None).await?; + let last_commit = &commit_info[0]; + let parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["targetSize"], json!("2000000")); + assert_eq!(parameters["predicate"], "[]"); Ok(()) } @@ -236,7 +240,19 @@ async fn test_optimize_with_partitions() -> Result<(), Box> { assert_eq!(version + 1, dt.version()); assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); - assert_eq!(dt.get_state().files().len(), 3); + assert_eq!(dt.get_files_count(), 3); + + let partition_adds = dt + .get_active_add_actions_by_partitions(&filter)? + .collect::, _>>()?; + assert_eq!(partition_adds.len(), 1); + let partition_values = partition_adds[0].partition_values()?; + assert_eq!( + partition_values.get("date"), + Some(&deltalake_core::kernel::Scalar::String( + "2022-05-22".to_string() + )) + ); Ok(()) } @@ -269,7 +285,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; let plan = create_merge_plan( OptimizeType::Compact, - &dt.state, + dt.snapshot()?, &filter, None, WriterProperties::builder().build(), @@ -277,36 +293,25 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let other_dt = deltalake_core::open_table(uri).await?; - let add = &other_dt.get_state().files()[0]; - let remove = Remove { - path: add.path.clone(), - deletion_timestamp: Some( - SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as i64, - ), - data_change: true, - extended_file_metadata: None, - size: Some(add.size), - partition_values: Some(add.partition_values.clone()), - tags: Some(HashMap::new()), - deletion_vector: add.deletion_vector.clone(), - base_row_id: add.base_row_id, - default_row_commit_version: add.default_row_commit_version, - }; + let add = &other_dt.snapshot()?.log_data().into_iter().next().unwrap(); + let remove = add.remove_action(true); let operation = DeltaOperation::Delete { predicate: None }; - commit( - other_dt.log_store().as_ref(), - &vec![Action::Remove(remove)], - operation, - &other_dt.state, - None, - ) - .await?; + CommitBuilder::default() + .with_actions(vec![Action::Remove(remove)]) + .build(Some(other_dt.snapshot()?), other_dt.log_store(), operation)? + .await?; - let maybe_metrics = plan.execute(dt.log_store(), &dt.state, 1, 20, None).await; + let maybe_metrics = plan + .execute( + dt.log_store(), + dt.snapshot()?, + 1, + 20, + None, + CommitProperties::default(), + ) + .await; assert!(maybe_metrics.is_err()); assert_eq!(dt.version(), version + 1); @@ -339,7 +344,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; let plan = create_merge_plan( OptimizeType::Compact, - &dt.state, + dt.snapshot()?, &filter, None, WriterProperties::builder().build(), @@ -355,7 +360,16 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { ) .await?; - let metrics = plan.execute(dt.log_store(), &dt.state, 1, 20, None).await?; + let metrics = plan + .execute( + dt.log_store(), + dt.snapshot()?, + 1, + 20, + None, + CommitProperties::default(), + ) + .await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); @@ -387,7 +401,7 @@ async fn test_commit_interval() -> Result<(), Box> { let plan = create_merge_plan( OptimizeType::Compact, - &dt.state, + dt.snapshot()?, &[], None, WriterProperties::builder().build(), @@ -396,10 +410,11 @@ async fn test_commit_interval() -> Result<(), Box> { let metrics = plan .execute( dt.log_store(), - &dt.state, + dt.snapshot()?, 1, 20, Some(Duration::from_secs(0)), // this will cause as many commits as num_files_added + CommitProperties::default(), ) .await?; assert_eq!(metrics.num_files_added, 2); @@ -411,7 +426,6 @@ async fn test_commit_interval() -> Result<(), Box> { } #[tokio::test] -#[ignore] /// Validate that bin packing is idempotent. async fn test_idempotent() -> Result<(), Box> { //TODO: Compression makes it hard to get the target file size... @@ -602,10 +616,10 @@ async fn test_commit_info() -> Result<(), Box> { .optimize() .with_target_size(2_000_000) .with_filters(&filter); - let (mut dt, metrics) = optimize.await?; + let (dt, metrics) = optimize.await?; let commit_info = dt.history(None).await?; - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let commit_metrics = serde_json::from_value::(last_commit.info["operationMetrics"].clone())?; @@ -614,8 +628,7 @@ async fn test_commit_info() -> Result<(), Box> { assert_eq!(last_commit.read_version, Some(version)); let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["targetSize"], json!("2000000")); - // TODO: Requires a string representation for PartitionFilter - // assert_eq!(parameters["predicate"], None); + assert_eq!(parameters["predicate"], "[\"date = '2022-05-22'\"]"); Ok(()) } @@ -714,7 +727,7 @@ async fn test_zorder_unpartitioned() -> Result<(), Box> { assert_eq!(metrics.total_considered_files, 2); // Check data - let files = dt.get_files(); + let files = dt.get_files_iter()?.collect::>(); assert_eq!(files.len(), 1); let actual = read_parquet_file(&files[0], dt.object_store()).await?; diff --git a/crates/deltalake-core/tests/command_restore.rs b/crates/core/tests/command_restore.rs similarity index 78% rename from crates/deltalake-core/tests/command_restore.rs rename to crates/core/tests/command_restore.rs index 80c2083261..1e49132d23 100644 --- a/crates/deltalake-core/tests/command_restore.rs +++ b/crates/core/tests/command_restore.rs @@ -1,17 +1,18 @@ -#![cfg(all(feature = "arrow", feature = "parquet", feature = "datafusion"))] - use arrow::datatypes::Schema as ArrowSchema; use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType as ArrowDataType, Field}; use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; use deltalake_core::kernel::{DataType, PrimitiveType, StructField}; use deltalake_core::protocol::SaveMode; +use deltalake_core::storage::commit_uri_from_version; use deltalake_core::{DeltaOps, DeltaTable}; use rand::Rng; use std::error::Error; use std::fs; use std::sync::Arc; -use tempdir::TempDir; +use std::thread; +use std::time::Duration; +use tempfile::TempDir; #[derive(Debug)] struct Context { @@ -33,7 +34,7 @@ async fn setup_test() -> Result> { ), ]; - let tmp_dir = tempdir::TempDir::new("restore_table").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let table_uri = tmp_dir.path().to_str().to_owned().unwrap(); let table = DeltaOps::try_from_uri(table_uri) .await? @@ -42,19 +43,21 @@ async fn setup_test() -> Result> { .await?; let batch = get_record_batch(); - + thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) .write(vec![batch.clone()]) .with_save_mode(SaveMode::Append) .await .unwrap(); + thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) .write(vec![batch.clone()]) .with_save_mode(SaveMode::Overwrite) .await .unwrap(); + thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) .write(vec![batch.clone()]) .with_save_mode(SaveMode::Append) @@ -96,26 +99,35 @@ async fn test_restore_by_version() -> Result<(), Box> { let result = DeltaOps(table).restore().with_version_to_restore(1).await?; assert_eq!(result.1.num_restored_file, 1); assert_eq!(result.1.num_removed_file, 2); - assert_eq!(result.0.state.version(), 4); + assert_eq!(result.0.snapshot()?.version(), 4); let table_uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let mut table = DeltaOps::try_from_uri(table_uri).await?; table.0.load_version(1).await?; - assert_eq!(table.0.state.files(), result.0.state.files()); + assert_eq!( + table.0.snapshot()?.file_actions()?, + result.0.snapshot()?.file_actions()? + ); let result = DeltaOps(result.0) .restore() .with_version_to_restore(0) .await?; - assert_eq!(result.0.state.files().len(), 0); + assert_eq!(result.0.get_files_count(), 0); Ok(()) } #[tokio::test] async fn test_restore_by_datetime() -> Result<(), Box> { let context = setup_test().await?; - let mut table = context.table; - let history = table.history(Some(10)).await?; - let timestamp = history.get(1).unwrap().timestamp.unwrap(); + let table = context.table; + let version = 1; + + // The way we obtain a timestamp for a version will have to change when/if we start using CommitInfo for timestamps + let meta = table + .object_store() + .head(&commit_uri_from_version(version)) + .await?; + let timestamp = meta.last_modified.timestamp_millis(); let naive = NaiveDateTime::from_timestamp_millis(timestamp).unwrap(); let datetime: DateTime = Utc.from_utc_datetime(&naive); @@ -125,14 +137,14 @@ async fn test_restore_by_datetime() -> Result<(), Box> { .await?; assert_eq!(result.1.num_restored_file, 1); assert_eq!(result.1.num_removed_file, 2); - assert_eq!(result.0.state.version(), 4); + assert_eq!(result.0.snapshot()?.version(), 4); Ok(()) } #[tokio::test] async fn test_restore_with_error_params() -> Result<(), Box> { let context = setup_test().await?; - let mut table = context.table; + let table = context.table; let history = table.history(Some(10)).await?; let timestamp = history.get(1).unwrap().timestamp.unwrap(); let naive = NaiveDateTime::from_timestamp_millis(timestamp).unwrap(); @@ -158,12 +170,17 @@ async fn test_restore_with_error_params() -> Result<(), Box> { async fn test_restore_file_missing() -> Result<(), Box> { let context = setup_test().await?; - for file in context.table.state.files().iter() { - let p = context.tmp_dir.path().join(file.clone().path); + for file in context.table.snapshot()?.log_data() { + let p = context.tmp_dir.path().join(file.path().as_ref()); fs::remove_file(p).unwrap(); } - for file in context.table.state.all_tombstones().iter() { + for file in context + .table + .snapshot()? + .all_tombstones(context.table.object_store().clone()) + .await? + { let p = context.tmp_dir.path().join(file.clone().path); fs::remove_file(p).unwrap(); } @@ -180,12 +197,17 @@ async fn test_restore_file_missing() -> Result<(), Box> { async fn test_restore_allow_file_missing() -> Result<(), Box> { let context = setup_test().await?; - for file in context.table.state.files().iter() { - let p = context.tmp_dir.path().join(file.clone().path); + for file in context.table.snapshot()?.log_data() { + let p = context.tmp_dir.path().join(file.path().as_ref()); fs::remove_file(p).unwrap(); } - for file in context.table.state.all_tombstones().iter() { + for file in context + .table + .snapshot()? + .all_tombstones(context.table.object_store().clone()) + .await? + { let p = context.tmp_dir.path().join(file.clone().path); fs::remove_file(p).unwrap(); } diff --git a/crates/deltalake-core/tests/command_vacuum.rs b/crates/core/tests/command_vacuum.rs similarity index 97% rename from crates/deltalake-core/tests/command_vacuum.rs rename to crates/core/tests/command_vacuum.rs index 51ff3217b3..54ec03cdb2 100644 --- a/crates/deltalake-core/tests/command_vacuum.rs +++ b/crates/core/tests/command_vacuum.rs @@ -1,15 +1,13 @@ use chrono::Duration; -use common::clock::TestClock; -use common::TestContext; use deltalake_core::kernel::StructType; use deltalake_core::operations::vacuum::Clock; use deltalake_core::operations::DeltaOps; +use deltalake_test::clock::TestClock; +use deltalake_test::*; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use serde_json::json; use std::sync::Arc; -mod common; - /// Basic schema pub fn get_xy_date_schema() -> StructType { serde_json::from_value(json!({ @@ -51,7 +49,7 @@ async fn test_non_partitioned_table() { ]; for path in paths { - common::add_file( + add_file( &mut table, &path, "random junk".as_bytes().into(), @@ -64,7 +62,7 @@ async fn test_non_partitioned_table() { clock.tick(Duration::seconds(10)); - common::remove_file( + remove_file( &mut table, "delete_me.parquet", &[], @@ -103,7 +101,7 @@ async fn test_partitioned_table() { let partition_values = [("date", Some("2022-07-03")), ("x", Some("2"))]; for path in paths { - common::add_file( + add_file( &mut table, &path, "random junk".as_bytes().into(), @@ -116,7 +114,7 @@ async fn test_partitioned_table() { clock.tick(Duration::seconds(10)); - common::remove_file( + remove_file( &mut table, "date=2022-07-03/x=2/delete_me.parquet", &partition_values, @@ -168,7 +166,7 @@ async fn test_partitions_included() { let partition_values = &[("_date", Some("2022-07-03"))]; for path in paths { - common::add_file( + add_file( &mut table, &path, "random junk".as_bytes().into(), @@ -181,7 +179,7 @@ async fn test_partitions_included() { clock.tick(Duration::seconds(10)); - common::remove_file( + remove_file( &mut table, "_date=2022-07-03/delete_me.parquet", partition_values, @@ -247,7 +245,7 @@ async fn test_non_managed_files() { ]; for path in paths_delete.iter().chain(paths_ignore.iter()) { - common::add_file( + add_file( &mut table, path, "random junk".as_bytes().into(), diff --git a/crates/deltalake-core/tests/commit_info_format.rs b/crates/core/tests/commit_info_format.rs similarity index 73% rename from crates/deltalake-core/tests/commit_info_format.rs rename to crates/core/tests/commit_info_format.rs index a9d05e4c11..d6ab6740ad 100644 --- a/crates/deltalake-core/tests/commit_info_format.rs +++ b/crates/core/tests/commit_info_format.rs @@ -2,15 +2,14 @@ mod fs_common; use deltalake_core::kernel::Action; -use deltalake_core::operations::transaction::commit; +use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use serde_json::json; use std::error::Error; -use tempdir::TempDir; #[tokio::test] async fn test_operational_parameters() -> Result<(), Box> { - let path = TempDir::new("operational_parameters").unwrap(); + let path = tempfile::tempdir().unwrap(); let mut table = fs_common::create_table(path.path().to_str().unwrap(), None).await; let add = fs_common::add(0); @@ -21,18 +20,14 @@ async fn test_operational_parameters() -> Result<(), Box> { predicate: None, }; - commit( - table.log_store().as_ref(), - &actions, - operation, - &table.state, - None, - ) - .await?; + CommitBuilder::default() + .with_actions(actions) + .build(Some(table.snapshot()?), table.log_store(), operation)? + .await?; table.update().await?; let commit_info = table.history(None).await?; - let last_commit = &commit_info[commit_info.len() - 1]; + let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); assert_eq!(parameters["mode"], json!("Append")); assert_eq!(parameters["partitionBy"], json!("[\"some_partition\"]")); diff --git a/crates/deltalake-core/tests/fs_common/mod.rs b/crates/core/tests/fs_common/mod.rs similarity index 87% rename from crates/deltalake-core/tests/fs_common/mod.rs rename to crates/core/tests/fs_common/mod.rs index c79fc833da..25da7ca321 100644 --- a/crates/deltalake-core/tests/fs_common/mod.rs +++ b/crates/core/tests/fs_common/mod.rs @@ -3,13 +3,12 @@ use deltalake_core::kernel::{ Action, Add, DataType, PrimitiveType, Remove, StructField, StructType, }; use deltalake_core::operations::create::CreateBuilder; -use deltalake_core::operations::transaction::commit; +use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; -use deltalake_core::storage::config::configure_store; use deltalake_core::storage::{GetResult, ObjectStoreResult}; use deltalake_core::DeltaTable; use object_store::path::Path as StorePath; -use object_store::ObjectStore; +use object_store::{ObjectStore, PutOptions, PutResult}; use serde_json::Value; use std::collections::HashMap; use std::fs; @@ -37,7 +36,7 @@ pub async fn create_table_from_json( partition_columns: Vec<&str>, config: Value, ) -> DeltaTable { - assert!(path.starts_with("./tests/data")); + assert!(path.starts_with("../test/tests/data")); std::fs::create_dir_all(path).unwrap(); std::fs::remove_dir_all(path).unwrap(); std::fs::create_dir_all(path).unwrap(); @@ -85,7 +84,6 @@ pub fn add(offset_millis: i64) -> Add { path: Uuid::new_v4().to_string(), size: 100, partition_values: Default::default(), - partition_values_parsed: None, modification_time: Utc::now().timestamp_millis() - offset_millis, data_change: true, stats: None, @@ -121,15 +119,17 @@ pub async fn commit_actions( actions: Vec, operation: DeltaOperation, ) -> i64 { - let version = commit( - table.log_store().as_ref(), - &actions, - operation, - &table.state, - None, - ) - .await - .unwrap(); + let version = CommitBuilder::default() + .with_actions(actions) + .build( + Some(table.snapshot().unwrap()), + table.log_store().clone(), + operation, + ) + .unwrap() + .await + .unwrap() + .version(); table.update().await.unwrap(); version } @@ -144,15 +144,14 @@ impl std::fmt::Display for SlowStore { } } -#[allow(dead_code)] impl SlowStore { + #[allow(dead_code)] pub fn new( location: Url, - options: impl Into + Clone, + _options: impl Into + Clone, ) -> deltalake_core::DeltaResult { - let mut options = options.into(); Ok(Self { - inner: configure_store(&location, &mut options).unwrap(), + inner: deltalake_core::storage::store_for(&location)?, }) } } @@ -160,10 +159,19 @@ impl SlowStore { #[async_trait::async_trait] impl ObjectStore for SlowStore { /// Save the provided bytes to the specified location. - async fn put(&self, location: &StorePath, bytes: bytes::Bytes) -> ObjectStoreResult<()> { + async fn put(&self, location: &StorePath, bytes: bytes::Bytes) -> ObjectStoreResult { self.inner.put(location, bytes).await } + async fn put_opts( + &self, + location: &StorePath, + bytes: bytes::Bytes, + options: PutOptions, + ) -> ObjectStoreResult { + self.inner.put_opts(location, bytes, options).await + } + /// Return the bytes that are stored at the specified location. async fn get(&self, location: &StorePath) -> ObjectStoreResult { tokio::time::sleep(tokio::time::Duration::from_secs_f64(0.01)).await; @@ -205,27 +213,23 @@ impl ObjectStore for SlowStore { /// /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of /// `foo/bar_baz/x`. - async fn list( + fn list( &self, prefix: Option<&StorePath>, - ) -> ObjectStoreResult< - futures::stream::BoxStream<'_, ObjectStoreResult>, - > { - self.inner.list(prefix).await + ) -> futures::stream::BoxStream<'_, ObjectStoreResult> { + self.inner.list(prefix) } /// List all the objects with the given prefix and a location greater than `offset` /// /// Some stores, such as S3 and GCS, may be able to push `offset` down to reduce /// the number of network requests required - async fn list_with_offset( + fn list_with_offset( &self, prefix: Option<&StorePath>, offset: &StorePath, - ) -> ObjectStoreResult< - futures::stream::BoxStream<'_, ObjectStoreResult>, - > { - self.inner.list_with_offset(prefix, offset).await + ) -> futures::stream::BoxStream<'_, ObjectStoreResult> { + self.inner.list_with_offset(prefix, offset) } /// List objects with the given prefix and an implementation specific diff --git a/crates/core/tests/integration.rs b/crates/core/tests/integration.rs new file mode 100644 index 0000000000..1dbd28f68a --- /dev/null +++ b/crates/core/tests/integration.rs @@ -0,0 +1,76 @@ +use deltalake_test::read::read_table_paths; +use deltalake_test::utils::*; +use deltalake_test::{test_concurrent_writes, test_read_tables}; +use object_store::path::Path; +use serial_test::serial; + +#[allow(dead_code)] +mod fs_common; + +static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; + +#[tokio::test] +#[serial] +async fn test_integration_local() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + + test_read_tables(&context).await?; + + for prefix in TEST_PREFIXES { + read_table_paths(&context, prefix, prefix).await?; + } + + Ok(()) +} + +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] +#[serial] +async fn test_concurrency_local() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + + test_concurrent_writes(&context).await?; + + Ok(()) +} + +#[tokio::test] +async fn test_action_reconciliation() { + let path = "./tests/data/action_reconciliation"; + let mut table = fs_common::create_table(path, None).await; + + // Add a file. + let a = fs_common::add(3 * 60 * 1000); + assert_eq!(1, fs_common::commit_add(&mut table, &a).await); + assert_eq!( + table.get_files_iter().unwrap().collect::>(), + vec![Path::from(a.path.clone())] + ); + + // Remove added file. + let r = deltalake_core::kernel::Remove { + path: a.path.clone(), + deletion_timestamp: Some(chrono::Utc::now().timestamp_millis()), + data_change: false, + extended_file_metadata: None, + partition_values: None, + size: None, + tags: None, + deletion_vector: None, + base_row_id: None, + default_row_commit_version: None, + }; + + assert_eq!(2, fs_common::commit_removes(&mut table, vec![&r]).await); + assert_eq!(table.get_files_iter().unwrap().count(), 0); + assert_eq!( + table + .snapshot() + .unwrap() + .all_tombstones(table.object_store().clone()) + .await + .unwrap() + .map(|r| r.path.clone()) + .collect::>(), + vec![a.path.clone()] + ); +} diff --git a/crates/deltalake-core/tests/integration_checkpoint.rs b/crates/core/tests/integration_checkpoint.rs similarity index 82% rename from crates/deltalake-core/tests/integration_checkpoint.rs rename to crates/core/tests/integration_checkpoint.rs index 56d253eb85..ce4525ba83 100644 --- a/crates/deltalake-core/tests/integration_checkpoint.rs +++ b/crates/core/tests/integration_checkpoint.rs @@ -3,54 +3,20 @@ use chrono::Utc; use deltalake_core::checkpoints::{cleanup_expired_logs_for, create_checkpoint}; use deltalake_core::kernel::{DataType, PrimitiveType}; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult}; use deltalake_core::writer::{DeltaWriter, JsonWriter}; use deltalake_core::{errors::DeltaResult, DeltaOps, DeltaTableBuilder, ObjectStore}; +use deltalake_test::utils::*; use object_store::path::Path; use serde_json::json; use serial_test::serial; use std::time::Duration; use tokio::time::sleep; -#[tokio::test] -async fn cleanup_metadata_fs_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Local)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[tokio::test] -#[serial] -async fn cleanup_metadata_aws_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Amazon)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn cleanup_metadata_azure_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Microsoft)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(feature = "gcs")] #[tokio::test] #[serial] -async fn cleanup_metadata_gcp_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Google)?; - cleanup_metadata_test(&context).await?; - Ok(()) -} - -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn cleanup_metadata_hdfs_test() -> TestResult { - let context = IntegrationContext::new(StorageIntegration::Hdfs)?; +async fn cleanup_metadata_fs_test() -> TestResult { + let storage = Box::new(LocalStorageIntegration::default()); + let context = IntegrationContext::new(storage)?; cleanup_metadata_test(&context).await?; Ok(()) } @@ -109,6 +75,7 @@ async fn cleanup_metadata_test(context: &IntegrationContext) -> TestResult { } #[tokio::test] +#[serial] async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { let _ = std::fs::remove_dir_all("./tests/data/issue_1420"); diff --git a/crates/deltalake-core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs similarity index 89% rename from crates/deltalake-core/tests/integration_datafusion.rs rename to crates/core/tests/integration_datafusion.rs index 51be591071..4be66534fe 100644 --- a/crates/deltalake-core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -1,8 +1,8 @@ -#![cfg(all(feature = "integration_test", feature = "datafusion"))] +#![cfg(feature = "datafusion")] use arrow::array::Int64Array; -use common::datafusion::context_with_delta_table_factory; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; +use deltalake_test::datafusion::*; +use deltalake_test::utils::*; use serial_test::serial; use std::collections::{HashMap, HashSet}; @@ -32,28 +32,29 @@ use url::Url; use deltalake_core::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; use deltalake_core::kernel::{DataType, MapType, PrimitiveType, StructField, StructType}; +use deltalake_core::logstore::logstore_for; use deltalake_core::operations::create::CreateBuilder; use deltalake_core::protocol::SaveMode; use deltalake_core::writer::{DeltaWriter, RecordBatchWriter}; use deltalake_core::{ open_table, operations::{write::WriteBuilder, DeltaOps}, - storage::config::configure_log_store, DeltaTable, DeltaTableError, }; use std::error::Error; -mod common; - mod local { use datafusion::common::stats::Precision; - use deltalake_core::writer::JsonWriter; + use deltalake_core::{logstore::default_logstore, writer::JsonWriter}; + use object_store::local::LocalFileSystem; use super::*; #[tokio::test] #[serial] async fn test_datafusion_local() -> TestResult { - test_datafusion(StorageIntegration::Local).await + let storage = Box::::default(); + let context = IntegrationContext::new(storage)?; + test_datafusion(&context).await } fn get_scanned_files(node: &dyn ExecutionPlan) -> HashSet(self, mut seq: A) -> Result - where - A: SeqAccess<'de>, - { - let state = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let config = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let storage_config: LogStoreConfig = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let log_store = configure_log_store( - storage_config.location.as_ref(), - storage_config.options, - None, - ) - .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; - let last_check_point = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let version_timestamp = seq - .next_element()? - .ok_or_else(|| A::Error::invalid_length(0, &self))?; - - let table = DeltaTable { - state, - config, - log_store, - last_check_point, - version_timestamp, - }; - Ok(table) - } - } - - deserializer.deserialize_seq(DeltaTableVisitor {}) - } -} - -impl DeltaTable { - /// Create a new Delta Table struct without loading any data from backing storage. - /// - /// NOTE: This is for advanced users. If you don't know why you need to use this method, please - /// call one of the `open_table` helper methods instead. - pub fn new(log_store: LogStoreRef, config: DeltaTableConfig) -> Self { - Self { - state: DeltaTableState::with_version(-1), - log_store, - config, - last_check_point: None, - version_timestamp: HashMap::new(), - } - } - - /// Create a new [`DeltaTable`] from a [`DeltaTableState`] without loading any - /// data from backing storage. - /// - /// NOTE: This is for advanced users. If you don't know why you need to use this method, - /// please call one of the `open_table` helper methods instead. - pub(crate) fn new_with_state(log_store: LogStoreRef, state: DeltaTableState) -> Self { - Self { - state, - log_store, - config: Default::default(), - last_check_point: None, - version_timestamp: HashMap::new(), - } - } - - /// get a shared reference to the delta object store - pub fn object_store(&self) -> ObjectStoreRef { - self.log_store.object_store() - } - - /// The URI of the underlying data - pub fn table_uri(&self) -> String { - self.log_store.root_uri() - } - - /// get a shared reference to the log store - pub fn log_store(&self) -> LogStoreRef { - self.log_store.clone() - } - - /// Return the list of paths of given checkpoint. - pub fn get_checkpoint_data_paths(&self, check_point: &CheckPoint) -> Vec { - let checkpoint_prefix = format!("{:020}", check_point.version); - let log_path = self.log_store.log_path(); - let mut checkpoint_data_paths = Vec::new(); - - match check_point.parts { - None => { - let path = log_path.child(&*format!("{checkpoint_prefix}.checkpoint.parquet")); - checkpoint_data_paths.push(path); - } - Some(parts) => { - for i in 0..parts { - let path = log_path.child(&*format!( - "{}.checkpoint.{:010}.{:010}.parquet", - checkpoint_prefix, - i + 1, - parts - )); - checkpoint_data_paths.push(path); - } - } - } - - checkpoint_data_paths - } - - /// This method scans delta logs to find the earliest delta log version - async fn get_earliest_delta_log_version(&self) -> Result { - // TODO check if regex matches against path - lazy_static! { - static ref DELTA_LOG_REGEX: Regex = - Regex::new(r"^_delta_log/(\d{20})\.(json|checkpoint)*$").unwrap(); - } - - let mut current_delta_log_ver = i64::MAX; - - // Get file objects from table. - let storage = self.object_store(); - let mut stream = storage.list(Some(self.log_store.log_path())).await?; - while let Some(obj_meta) = stream.next().await { - let obj_meta = obj_meta?; - - if let Some(captures) = DELTA_LOG_REGEX.captures(obj_meta.location.as_ref()) { - let log_ver_str = captures.get(1).unwrap().as_str(); - let log_ver: i64 = log_ver_str.parse().unwrap(); - if log_ver < current_delta_log_ver { - current_delta_log_ver = log_ver; - } - } - } - Ok(current_delta_log_ver) - } - - #[cfg(any(feature = "parquet", feature = "parquet2"))] - async fn restore_checkpoint(&mut self, check_point: CheckPoint) -> Result<(), DeltaTableError> { - self.state = DeltaTableState::from_checkpoint(self, &check_point).await?; - - Ok(()) - } - - /// returns the latest available version of the table - pub async fn get_latest_version(&self) -> Result { - self.log_store.get_latest_version(self.version()).await - } - - /// Currently loaded version of the table - pub fn version(&self) -> i64 { - self.state.version() - } - - /// Load DeltaTable with data from latest checkpoint - pub async fn load(&mut self) -> Result<(), DeltaTableError> { - self.last_check_point = None; - self.state = DeltaTableState::with_version(-1); - self.update().await - } - - /// Get the list of actions for the next commit - pub async fn peek_next_commit( - &self, - current_version: i64, - ) -> Result { - let next_version = current_version + 1; - let commit_log_bytes = match self.log_store.read_commit_entry(next_version).await { - Ok(Some(bytes)) => Ok(bytes), - Ok(None) => return Ok(PeekCommit::UpToDate), - Err(err) => Err(err), - }?; - - let actions = logstore::get_actions(next_version, commit_log_bytes).await; - Ok(PeekCommit::New(next_version, actions.unwrap())) - } - - /// Updates the DeltaTable to the most recent state committed to the transaction log by - /// loading the last checkpoint and incrementally applying each version since. - #[cfg(any(feature = "parquet", feature = "parquet2"))] - pub async fn update(&mut self) -> Result<(), DeltaTableError> { - match get_last_checkpoint(self.log_store.as_ref()).await { - Ok(last_check_point) => { - debug!("update with latest checkpoint {last_check_point:?}"); - if Some(last_check_point) == self.last_check_point { - self.update_incremental(None).await - } else { - self.last_check_point = Some(last_check_point); - self.restore_checkpoint(last_check_point).await?; - self.update_incremental(None).await - } - } - Err(ProtocolError::CheckpointNotFound) => { - debug!("update without checkpoint"); - self.update_incremental(None).await - } - Err(err) => Err(DeltaTableError::from(err)), - } - } - - /// Updates the DeltaTable to the most recent state committed to the transaction log. - #[cfg(not(any(feature = "parquet", feature = "parquet2")))] - pub async fn update(&mut self) -> Result<(), DeltaTableError> { - self.update_incremental(None).await - } - - /// Updates the DeltaTable to the latest version by incrementally applying newer versions. - /// It assumes that the table is already updated to the current version `self.version`. - pub async fn update_incremental( - &mut self, - max_version: Option, - ) -> Result<(), DeltaTableError> { - debug!( - "incremental update with version({}) and max_version({max_version:?})", - self.version(), - ); - - // update to latest version if given max_version is not larger than current version - let max_version = max_version.filter(|x| x > &self.version()); - let max_version: i64 = match max_version { - Some(x) => x, - None => self.get_latest_version().await?, - }; - - let buf_size = self.config.log_buffer_size; - - let log_store = self.log_store.clone(); - let mut log_stream = futures::stream::iter(self.version() + 1..max_version + 1) - .map(|version| { - let log_store = log_store.clone(); - async move { - if let Some(data) = log_store.read_commit_entry(version).await? { - Ok(Some((version, logstore::get_actions(version, data).await?))) - } else { - Ok(None) - } - } - }) - .buffered(buf_size); - - while let Some(res) = log_stream.next().await { - let (new_version, actions) = match res { - Ok(Some((version, actions))) => (version, actions), - Ok(None) => break, - Err(err) => return Err(err), - }; - - debug!("merging table state with version: {new_version}"); - let s = DeltaTableState::from_actions(actions, new_version)?; - self.state - .merge(s, self.config.require_tombstones, self.config.require_files); - if self.version() == max_version { - return Ok(()); - } - } - - if self.version() == -1 { - return Err(DeltaTableError::not_a_table(self.table_uri())); - } - - Ok(()) - } - - /// Loads the DeltaTable state for the given version. - pub async fn load_version(&mut self, version: i64) -> Result<(), DeltaTableError> { - // check if version is valid - let commit_uri = commit_uri_from_version(version); - match self.object_store().head(&commit_uri).await { - Ok(_) => {} - Err(ObjectStoreError::NotFound { .. }) => { - return Err(DeltaTableError::InvalidVersion(version)); - } - Err(e) => { - return Err(DeltaTableError::from(e)); - } - } - - // 1. find latest checkpoint below version - #[cfg(any(feature = "parquet", feature = "parquet2"))] - match find_latest_check_point_for_version(self.log_store.as_ref(), version).await? { - Some(check_point) => { - self.restore_checkpoint(check_point).await?; - } - None => { - // no checkpoint found, clear table state and start from the beginning - self.state = DeltaTableState::with_version(-1); - } - } - - debug!("update incrementally from version {version}"); - // 2. apply all logs starting from checkpoint - self.update_incremental(Some(version)).await?; - - Ok(()) - } - - pub(crate) async fn get_version_timestamp( - &mut self, - version: i64, - ) -> Result { - match self.version_timestamp.get(&version) { - Some(ts) => Ok(*ts), - None => { - let meta = self - .object_store() - .head(&commit_uri_from_version(version)) - .await?; - let ts = meta.last_modified.timestamp(); - // also cache timestamp for version - self.version_timestamp.insert(version, ts); - - Ok(ts) - } - } - } - - /// Returns provenance information, including the operation, user, and so on, for each write to a table. - /// The table history retention is based on the `logRetentionDuration` property of the Delta Table, 30 days by default. - /// If `limit` is given, this returns the information of the latest `limit` commits made to this table. Otherwise, - /// it returns all commits from the earliest commit. - pub async fn history( - &mut self, - limit: Option, - ) -> Result, DeltaTableError> { - let mut version = match limit { - Some(l) => max(self.version() - l as i64 + 1, 0), - None => self.get_earliest_delta_log_version().await?, - }; - let mut commit_infos_list = vec![]; - let mut earliest_commit: Option = None; - - loop { - match DeltaTableState::from_commit(self, version).await { - Ok(state) => { - commit_infos_list.append(state.commit_infos().clone().as_mut()); - version += 1; - } - Err(e) => { - match e { - ProtocolError::EndOfLog => { - if earliest_commit.is_none() { - earliest_commit = - Some(self.get_earliest_delta_log_version().await?); - }; - if let Some(earliest) = earliest_commit { - if version < earliest { - version = earliest; - continue; - } - } else { - version -= 1; - if version == -1 { - return Err(DeltaTableError::not_a_table(self.table_uri())); - } - } - } - _ => { - return Err(DeltaTableError::from(e)); - } - } - return Ok(commit_infos_list); - } - } - } - } - - /// Obtain Add actions for files that match the filter - pub fn get_active_add_actions_by_partitions<'a>( - &'a self, - filters: &'a [PartitionFilter], - ) -> Result + '_, DeltaTableError> { - self.state.get_active_add_actions_by_partitions(filters) - } - - /// Returns the file list tracked in current table state filtered by provided - /// `PartitionFilter`s. - pub fn get_files_by_partitions( - &self, - filters: &[PartitionFilter], - ) -> Result, DeltaTableError> { - Ok(self - .get_active_add_actions_by_partitions(filters)? - .map(|add| { - // Try to preserve percent encoding if possible - match Path::parse(&add.path) { - Ok(path) => path, - Err(_) => Path::from(add.path.as_ref()), - } - }) - .collect()) - } - - /// Return the file uris as strings for the partition(s) - pub fn get_file_uris_by_partitions( - &self, - filters: &[PartitionFilter], - ) -> Result, DeltaTableError> { - let files = self.get_files_by_partitions(filters)?; - Ok(files - .iter() - .map(|fname| self.log_store.to_uri(fname)) - .collect()) - } - - /// Returns an iterator of file names present in the loaded state - #[inline] - pub fn get_files_iter(&self) -> impl Iterator + '_ { - self.state.file_paths_iter() - } - - /// Returns a collection of file names present in the loaded state - #[deprecated(since = "0.17.0", note = "use get_files_iter() instead")] - #[inline] - pub fn get_files(&self) -> Vec { - self.state.file_paths_iter().collect() - } - - /// Returns file names present in the loaded state in HashSet - #[deprecated(since = "0.17.0", note = "use get_files_iter() instead")] - pub fn get_file_set(&self) -> HashSet { - self.state.file_paths_iter().collect() - } - - /// Returns a URIs for all active files present in the current table version. - pub fn get_file_uris(&self) -> impl Iterator + '_ { - self.state - .file_paths_iter() - .map(|path| self.log_store.to_uri(&path)) - } - - /// Returns statistics for files, in order - pub fn get_stats(&self) -> impl Iterator, DeltaTableError>> + '_ { - self.state.files().iter().map(|add| { - add.get_stats() - .map_err(|e| DeltaTableError::InvalidStatsJson { json_err: e }) - }) - } - - /// Returns partition values for files, in order - pub fn get_partition_values( - &self, - ) -> impl Iterator>> + '_ { - self.state.files().iter().map(|add| &add.partition_values) - } - - /// Returns the currently loaded state snapshot. - pub fn get_state(&self) -> &DeltaTableState { - &self.state - } - - /// Returns current table protocol - pub fn protocol(&self) -> &Protocol { - self.state.protocol() - } - - /// Returns the metadata associated with the loaded state. - pub fn metadata(&self) -> Result<&Metadata, DeltaTableError> { - Ok(self.state.metadata_action()?) - } - - /// Returns the metadata associated with the loaded state. - #[deprecated(since = "0.17.0", note = "use metadata() instead")] - pub fn get_metadata(&self) -> Result<&DeltaTableMetaData, DeltaTableError> { - self.state.metadata().ok_or(DeltaTableError::NoMetadata) - } - - /// Returns a vector of active tombstones (i.e. `Remove` actions present in the current delta log). - pub fn get_tombstones(&self) -> impl Iterator { - self.state.unexpired_tombstones() - } - - /// Returns the current version of the DeltaTable based on the loaded metadata. - pub fn get_app_transaction_version(&self) -> &HashMap { - self.state.app_transaction_version() - } - - /// Returns the minimum reader version supported by the DeltaTable based on the loaded - /// metadata. - #[deprecated(since = "0.17.0", note = "use protocol().min_reader_version instead")] - pub fn get_min_reader_version(&self) -> i32 { - self.state.protocol().min_reader_version - } - - /// Returns the minimum writer version supported by the DeltaTable based on the loaded - /// metadata. - #[deprecated(since = "0.17.0", note = "use protocol().min_writer_version instead")] - pub fn get_min_writer_version(&self) -> i32 { - self.state.protocol().min_writer_version - } - - /// Returns current supported reader features by this table - #[deprecated(since = "0.17.0", note = "use protocol().reader_features instead")] - pub fn get_reader_features(&self) -> Option<&HashSet> { - self.state.protocol().reader_features.as_ref() - } - - /// Returns current supported writer features by this table - #[deprecated(since = "0.17.0", note = "use protocol().writer_features instead")] - pub fn get_writer_features(&self) -> Option<&HashSet> { - self.state.protocol().writer_features.as_ref() - } - - /// Return table schema parsed from transaction log. Return None if table hasn't been loaded or - /// no metadata was found in the log. - pub fn schema(&self) -> Option<&StructType> { - self.state.schema() - } - - /// Return table schema parsed from transaction log. Return `DeltaTableError` if table hasn't - /// been loaded or no metadata was found in the log. - pub fn get_schema(&self) -> Result<&StructType, DeltaTableError> { - self.schema().ok_or(DeltaTableError::NoSchema) - } - - /// Return the tables configurations that are encapsulated in the DeltaTableStates currentMetaData field - #[deprecated( - since = "0.17.0", - note = "use metadata().configuration or get_state().table_config() instead" - )] - pub fn get_configurations(&self) -> Result<&HashMap>, DeltaTableError> { - Ok(self - .state - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .get_configuration()) - } - - /// Time travel Delta table to the latest version that's created at or before provided - /// `datetime` argument. - /// - /// Internally, this methods performs a binary search on all Delta transaction logs. - pub async fn load_with_datetime( - &mut self, - datetime: DateTime, - ) -> Result<(), DeltaTableError> { - let mut min_version = 0; - let mut max_version = self.get_latest_version().await?; - let mut version = min_version; - let target_ts = datetime.timestamp(); - - // binary search - while min_version <= max_version { - let pivot = (max_version + min_version) / 2; - version = pivot; - let pts = self.get_version_timestamp(pivot).await?; - - match pts.cmp(&target_ts) { - Ordering::Equal => { - break; - } - Ordering::Less => { - min_version = pivot + 1; - } - Ordering::Greater => { - max_version = pivot - 1; - version = max_version - } - } - } - - if version < 0 { - version = 0; - } - - self.load_version(version).await - } -} - -impl fmt::Display for DeltaTable { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - writeln!(f, "DeltaTable({})", self.table_uri())?; - writeln!(f, "\tversion: {}", self.version())?; - match self.state.metadata() { - Some(metadata) => { - writeln!(f, "\tmetadata: {metadata}")?; - } - None => { - writeln!(f, "\tmetadata: None")?; - } - } - writeln!( - f, - "\tmin_version: read={}, write={}", - self.state.protocol().min_reader_version, - self.state.protocol().min_writer_version - )?; - writeln!(f, "\tfiles count: {}", self.state.files().len()) - } -} - -impl std::fmt::Debug for DeltaTable { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - write!(f, "DeltaTable <{}>", self.table_uri()) - } -} - -#[cfg(test)] -mod tests { - use pretty_assertions::assert_eq; - use tempdir::TempDir; - - use super::*; - use crate::kernel::{DataType, PrimitiveType, StructField}; - use crate::operations::create::CreateBuilder; - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - use crate::table::builder::DeltaTableBuilder; - - #[tokio::test] - async fn table_round_trip() { - let (dt, tmp_dir) = create_test_table().await; - let bytes = serde_json::to_vec(&dt).unwrap(); - let actual: DeltaTable = serde_json::from_slice(&bytes).unwrap(); - assert_eq!(actual.version(), dt.version()); - drop(tmp_dir); - } - - #[tokio::test] - async fn checkpoint_without_added_files_and_no_parts() { - let (dt, tmp_dir) = create_test_table().await; - let check_point = CheckPointBuilder::new(0, 0).build(); - let checkpoint_data_paths = dt.get_checkpoint_data_paths(&check_point); - assert_eq!(checkpoint_data_paths.len(), 1); - assert_eq!( - serde_json::to_string(&check_point).unwrap(), - "{\"version\":0,\"size\":0}" - ); - drop(tmp_dir); - } - - #[tokio::test] - async fn checkpoint_with_added_files() { - let num_of_file_added: i64 = 4; - let (dt, tmp_dir) = create_test_table().await; - let check_point = CheckPointBuilder::new(0, 0) - .with_num_of_add_files(num_of_file_added) - .build(); - let checkpoint_data_paths = dt.get_checkpoint_data_paths(&check_point); - assert_eq!(checkpoint_data_paths.len(), 1); - assert_eq!( - serde_json::to_string(&check_point).unwrap(), - "{\"version\":0,\"size\":0,\"num_of_add_files\":4}" - ); - drop(tmp_dir); - } - - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - #[test] - fn normalize_table_uri_s3() { - std::env::set_var("AWS_DEFAULT_REGION", "us-east-1"); - for table_uri in [ - "s3://tests/data/delta-0.8.0/", - "s3://tests/data/delta-0.8.0//", - "s3://tests/data/delta-0.8.0", - ] - .iter() - { - let table = DeltaTableBuilder::from_uri(table_uri).build().unwrap(); - assert_eq!(table.table_uri(), "s3://tests/data/delta-0.8.0"); - } - } - - 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"); - std::fs::create_dir(&table_dir).unwrap(); - - let dt = CreateBuilder::new() - .with_location(table_dir.to_str().unwrap()) - .with_table_name("Test Table Create") - .with_comment("This table is made to test the create function for a DeltaTable") - .with_columns(vec![ - StructField::new( - "Id".to_string(), - DataType::Primitive(PrimitiveType::Integer), - true, - ), - StructField::new( - "Name".to_string(), - DataType::Primitive(PrimitiveType::String), - true, - ), - ]) - .await - .unwrap(); - (dt, tmp_dir) - } -} diff --git a/crates/deltalake-core/src/table/state.rs b/crates/deltalake-core/src/table/state.rs deleted file mode 100644 index fa9078997c..0000000000 --- a/crates/deltalake-core/src/table/state.rs +++ /dev/null @@ -1,441 +0,0 @@ -//! The module for delta table state. - -use std::collections::HashMap; -use std::collections::HashSet; -use std::convert::TryFrom; -use std::io::{BufRead, BufReader, Cursor}; - -use chrono::Utc; -use lazy_static::lazy_static; -use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; -use serde::{Deserialize, Serialize}; - -use super::config::TableConfig; -use crate::errors::DeltaTableError; -use crate::kernel::{Action, Add, CommitInfo, DataType, DomainMetadata, Remove, StructType}; -use crate::kernel::{Metadata, Protocol}; -use crate::partitions::{DeltaTablePartition, PartitionFilter}; -use crate::protocol::ProtocolError; -use crate::storage::commit_uri_from_version; -use crate::table::DeltaTableMetaData; -use crate::DeltaTable; - -#[cfg(any(feature = "parquet", feature = "parquet2"))] -use super::{CheckPoint, DeltaTableConfig}; - -/// State snapshot currently held by the Delta Table instance. -#[derive(Default, Debug, Clone, Serialize, Deserialize)] -#[serde(rename_all = "camelCase")] -pub struct DeltaTableState { - // current table version represented by this table state - version: i64, - // A remove action should remain in the state of the table as a tombstone until it has expired. - // A tombstone expires when the creation timestamp of the delta file exceeds the expiration - tombstones: HashSet, - // active files for table state - files: Vec, - // Information added to individual commits - commit_infos: Vec, - // Domain metadatas provided by the system or user - domain_metadatas: Vec, - app_transaction_version: HashMap, - // table metadata corresponding to current version - current_metadata: Option, - protocol: Option, - metadata: Option, -} - -impl DeltaTableState { - /// Create Table state with specified version - pub fn with_version(version: i64) -> Self { - Self { - version, - ..Self::default() - } - } - - /// Return table version - pub fn version(&self) -> i64 { - self.version - } - - /// Construct a delta table state object from commit version. - pub async fn from_commit(table: &DeltaTable, version: i64) -> Result { - let commit_uri = commit_uri_from_version(version); - let commit_log_bytes = match table.object_store().get(&commit_uri).await { - Ok(get) => Ok(get.bytes().await?), - Err(ObjectStoreError::NotFound { .. }) => Err(ProtocolError::EndOfLog), - Err(source) => Err(ProtocolError::ObjectStore { source }), - }?; - let reader = BufReader::new(Cursor::new(commit_log_bytes)); - - let mut new_state = DeltaTableState::with_version(version); - for line in reader.lines() { - let action: Action = serde_json::from_str(line?.as_str())?; - new_state.process_action( - action, - table.config.require_tombstones, - table.config.require_files, - )?; - } - - Ok(new_state) - } - - /// Construct a delta table state object from a list of actions - pub fn from_actions(actions: Vec, version: i64) -> Result { - let mut new_state = DeltaTableState::with_version(version); - for action in actions { - new_state.process_action(action, true, true)?; - } - Ok(new_state) - } - - /// Update DeltaTableState with checkpoint data. - #[cfg(any(feature = "parquet", feature = "parquet2"))] - pub fn process_checkpoint_bytes( - &mut self, - data: bytes::Bytes, - table_config: &DeltaTableConfig, - ) -> Result<(), DeltaTableError> { - #[cfg(feature = "parquet")] - { - use parquet::file::reader::{FileReader, SerializedFileReader}; - - let preader = SerializedFileReader::new(data)?; - let schema = preader.metadata().file_metadata().schema(); - if !schema.is_group() { - return Err(DeltaTableError::from(ProtocolError::Generic( - "Action record in checkpoint should be a struct".to_string(), - ))); - } - for record in preader.get_row_iter(None)? { - self.process_action( - Action::from_parquet_record(schema, &record.unwrap())?, - table_config.require_tombstones, - table_config.require_files, - )?; - } - } - - #[cfg(feature = "parquet2")] - { - use crate::protocol::parquet2_read::actions_from_row_group; - use parquet2::read::read_metadata; - - let mut reader = std::io::Cursor::new(data); - let metadata = read_metadata(&mut reader)?; - - for row_group in metadata.row_groups { - for action in - actions_from_row_group(row_group, &mut reader).map_err(ProtocolError::from)? - { - self.process_action( - action, - table_config.require_tombstones, - table_config.require_files, - )?; - } - } - } - - Ok(()) - } - - /// Construct a delta table state object from checkpoint. - #[cfg(any(feature = "parquet", feature = "parquet2"))] - pub async fn from_checkpoint( - table: &DeltaTable, - check_point: &CheckPoint, - ) -> Result { - let checkpoint_data_paths = table.get_checkpoint_data_paths(check_point); - let mut new_state = Self::with_version(check_point.version); - - for f in &checkpoint_data_paths { - let obj = table.object_store().get(f).await?.bytes().await?; - new_state.process_checkpoint_bytes(obj, &table.config)?; - } - - Ok(new_state) - } - - /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec { - &self.commit_infos - } - - /// Full list of tombstones (remove actions) representing files removed from table state). - pub fn all_tombstones(&self) -> &HashSet { - &self.tombstones - } - - /// List of unexpired tombstones (remove actions) representing files removed from table state. - /// The retention period is set by `deletedFileRetentionDuration` with default value of 1 week. - pub fn unexpired_tombstones(&self) -> impl Iterator { - let retention_timestamp = Utc::now().timestamp_millis() - - self - .table_config() - .deleted_file_retention_duration() - .as_millis() as i64; - self.tombstones - .iter() - .filter(move |t| t.deletion_timestamp.unwrap_or(0) > retention_timestamp) - } - - /// Full list of add actions representing all parquet files that are part of the current - /// delta table state. - pub fn files(&self) -> &Vec { - self.files.as_ref() - } - - /// Returns an iterator of file names present in the loaded state - #[inline] - pub fn file_paths_iter(&self) -> impl Iterator + '_ { - self.files.iter().map(|add| match Path::parse(&add.path) { - Ok(path) => path, - Err(_) => Path::from(add.path.as_ref()), - }) - } - - /// HashMap containing the last txn version stored for every app id writing txn - /// actions. - pub fn app_transaction_version(&self) -> &HashMap { - &self.app_transaction_version - } - - /// The most recent protocol of the table. - pub fn protocol(&self) -> &Protocol { - lazy_static! { - static ref DEFAULT_PROTOCOL: Protocol = Protocol::default(); - } - self.protocol.as_ref().unwrap_or(&DEFAULT_PROTOCOL) - } - - /// The most recent metadata of the table. - pub fn metadata_action(&self) -> Result<&Metadata, ProtocolError> { - self.metadata.as_ref().ok_or(ProtocolError::NoMetaData) - } - - /// The most recent metadata of the table. - pub fn metadata(&self) -> Option<&DeltaTableMetaData> { - self.current_metadata.as_ref() - } - - /// The table schema - pub fn schema(&self) -> Option<&StructType> { - self.current_metadata.as_ref().map(|m| &m.schema) - } - - /// Well known table configuration - pub fn table_config(&self) -> TableConfig<'_> { - lazy_static! { - static ref DUMMY_CONF: HashMap> = HashMap::new(); - } - self.current_metadata - .as_ref() - .map(|meta| TableConfig(&meta.configuration)) - .unwrap_or_else(|| TableConfig(&DUMMY_CONF)) - } - - /// Merges new state information into our state - /// - /// The DeltaTableState also carries the version information for the given state, - /// as there is a one-to-one match between a table state and a version. In merge/update - /// scenarios we cannot infer the intended / correct version number. By default this - /// function will update the tracked version if the version on `new_state` is larger then the - /// currently set version however it is up to the caller to update the `version` field according - /// to the version the merged state represents. - pub fn merge( - &mut self, - mut new_state: DeltaTableState, - require_tombstones: bool, - require_files: bool, - ) { - if !new_state.tombstones.is_empty() { - self.files - .retain(|a| !new_state.tombstones.contains(a.path.as_str())); - } - - if require_tombstones && require_files { - new_state.tombstones.into_iter().for_each(|r| { - self.tombstones.insert(r); - }); - - if !new_state.files.is_empty() { - new_state.files.iter().for_each(|s| { - self.tombstones.remove(s.path.as_str()); - }); - } - } - - if require_files { - self.files.append(&mut new_state.files); - } - - if new_state.current_metadata.is_some() { - self.current_metadata = new_state.current_metadata.take(); - } - if new_state.metadata.is_some() { - self.metadata = new_state.metadata.take(); - } - - if new_state.protocol.is_some() { - self.protocol = new_state.protocol.take(); - } - - new_state - .app_transaction_version - .drain() - .for_each(|(app_id, version)| { - *self - .app_transaction_version - .entry(app_id) - .or_insert(version) = version - }); - - if !new_state.commit_infos.is_empty() { - self.commit_infos.append(&mut new_state.commit_infos); - } - - if self.version < new_state.version { - self.version = new_state.version - } - } - - /// Process given action by updating current state. - fn process_action( - &mut self, - action: Action, - require_tombstones: bool, - require_files: bool, - ) -> Result<(), ProtocolError> { - match action { - // TODO: optionally load CDC into TableState - Action::Cdc(_v) => {} - Action::Add(v) => { - if require_files { - self.files.push(v); - } - } - Action::Remove(v) => { - if require_tombstones && require_files { - self.tombstones.insert(v); - } - } - Action::Protocol(v) => { - self.protocol = Some(v); - } - Action::Metadata(v) => { - self.metadata = Some(v.clone()); - let md = DeltaTableMetaData::try_from(v)?; - self.current_metadata = Some(md); - } - Action::Txn(v) => { - *self - .app_transaction_version - .entry(v.app_id) - .or_insert(v.version) = v.version; - } - Action::CommitInfo(v) => { - self.commit_infos.push(v); - } - Action::DomainMetadata(v) => { - self.domain_metadatas.push(v); - } - } - - Ok(()) - } - - /// Obtain Add actions for files that match the filter - pub fn get_active_add_actions_by_partitions<'a>( - &'a self, - filters: &'a [PartitionFilter], - ) -> Result + '_, DeltaTableError> { - let current_metadata = self.metadata().ok_or(DeltaTableError::NoMetadata)?; - - let nonpartitioned_columns: Vec = filters - .iter() - .filter(|f| !current_metadata.partition_columns.contains(&f.key)) - .map(|f| f.key.to_string()) - .collect(); - - if !nonpartitioned_columns.is_empty() { - return Err(DeltaTableError::ColumnsNotPartitioned { - nonpartitioned_columns: { nonpartitioned_columns }, - }); - } - - let partition_col_data_types: HashMap<&String, &DataType> = current_metadata - .get_partition_col_data_types() - .into_iter() - .collect(); - - let actions = self.files().iter().filter(move |add| { - let partitions = add - .partition_values - .iter() - .map(|p| DeltaTablePartition::from_partition_value((p.0, p.1), "")) - .collect::>(); - filters - .iter() - .all(|filter| filter.match_partitions(&partitions, &partition_col_data_types)) - }); - Ok(actions) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::kernel::Txn; - use pretty_assertions::assert_eq; - - #[test] - fn state_round_trip() { - let expected = DeltaTableState { - version: 0, - tombstones: Default::default(), - files: vec![], - commit_infos: vec![], - domain_metadatas: vec![], - app_transaction_version: Default::default(), - current_metadata: None, - metadata: None, - protocol: None, - }; - let bytes = serde_json::to_vec(&expected).unwrap(); - let actual: DeltaTableState = serde_json::from_slice(&bytes).unwrap(); - assert_eq!(actual.version, expected.version); - } - - #[test] - fn state_records_new_txn_version() { - let mut app_transaction_version = HashMap::new(); - app_transaction_version.insert("abc".to_string(), 1); - app_transaction_version.insert("xyz".to_string(), 1); - - let mut state = DeltaTableState { - version: -1, - files: vec![], - commit_infos: vec![], - domain_metadatas: vec![], - tombstones: HashSet::new(), - current_metadata: None, - protocol: None, - metadata: None, - app_transaction_version, - }; - - let txn_action = Action::Txn(Txn { - app_id: "abc".to_string(), - version: 2, - last_updated: Some(0), - }); - - state.process_action(txn_action, false, true).unwrap(); - - assert_eq!(2, *state.app_transaction_version().get("abc").unwrap()); - assert_eq!(1, *state.app_transaction_version().get("xyz").unwrap()); - } -} diff --git a/crates/deltalake-core/src/test_utils.rs b/crates/deltalake-core/src/test_utils.rs deleted file mode 100644 index 190b720baf..0000000000 --- a/crates/deltalake-core/src/test_utils.rs +++ /dev/null @@ -1,644 +0,0 @@ -#![allow(dead_code, missing_docs)] -use crate::storage::utils::copy_table; -use crate::DeltaTableBuilder; -use chrono::Utc; -use fs_extra::dir::{copy, CopyOptions}; -use object_store::DynObjectStore; -use rand::Rng; -use serde_json::json; -use std::env; -use std::sync::Arc; -use tempdir::TempDir; - -pub type TestResult = Result<(), Box>; - -/// The IntegrationContext provides temporary resources to test against cloud storage services. -pub struct IntegrationContext { - pub integration: StorageIntegration, - bucket: String, - store: Arc, - tmp_dir: TempDir, -} - -impl IntegrationContext { - pub fn new( - integration: StorageIntegration, - ) -> Result> { - // environment variables are loaded from .env files if found. Otherwise - // default values based on the default setting of the respective emulators are set. - #[cfg(test)] - dotenvy::dotenv().ok(); - - integration.prepare_env(); - - let tmp_dir = TempDir::new("")?; - // create a fresh bucket in every context. THis is done via CLI... - let bucket = match integration { - StorageIntegration::Local => tmp_dir.as_ref().to_str().unwrap().to_owned(), - StorageIntegration::Onelake => { - let account_name = - env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap_or(String::from("onelake")); - let container_name = - env::var("AZURE_STORAGE_CONTAINER_NAME").unwrap_or(String::from("delta-rs")); - format!( - "{0}.dfs.fabric.microsoft.com/{1}", - account_name, container_name - ) - } - StorageIntegration::OnelakeAbfs => { - let account_name = - env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap_or(String::from("onelake")); - let container_name = - env::var("AZURE_STORAGE_CONTAINER_NAME").unwrap_or(String::from("delta-rs")); - format!( - "{0}@{1}.dfs.fabric.microsoft.com", - container_name, account_name - ) - } - _ => format!("test-delta-table-{}", Utc::now().timestamp()), - }; - - if let StorageIntegration::Google = integration { - gs_cli::prepare_env(); - let base_url = std::env::var("GOOGLE_BASE_URL")?; - let token = json!({"gcs_base_url": base_url, "disable_oauth": true, "client_email": "", "private_key": ""}); - let account_path = tmp_dir.path().join("gcs.json"); - std::fs::write(&account_path, serde_json::to_vec(&token)?)?; - set_env_if_not_set( - "GOOGLE_SERVICE_ACCOUNT", - account_path.as_path().to_str().unwrap(), - ); - } - - integration.create_bucket(&bucket)?; - let store_uri = match integration { - StorageIntegration::Amazon => format!("s3://{}", &bucket), - StorageIntegration::Microsoft => format!("az://{}", &bucket), - StorageIntegration::Onelake => format!("https://{}", &bucket), - StorageIntegration::OnelakeAbfs => format!("abfss://{}", &bucket), - StorageIntegration::Google => format!("gs://{}", &bucket), - StorageIntegration::Local => format!("file://{}", &bucket), - StorageIntegration::Hdfs => format!("hdfs://localhost:9000/{}", &bucket), - }; - // the "storage_backend" will always point to the root ofg the object store. - // TODO should we provide the store via object_Store builders? - let store = match integration { - StorageIntegration::Local => Arc::new( - object_store::local::LocalFileSystem::new_with_prefix(tmp_dir.path())?, - ), - _ => DeltaTableBuilder::from_uri(store_uri) - .with_allow_http(true) - .build_storage()? - .object_store(), - }; - - Ok(Self { - integration, - bucket, - store, - tmp_dir, - }) - } - - /// Get a a reference to the root object store - pub fn object_store(&self) -> Arc { - self.store.clone() - } - - /// Get the URI for initializing a store at the root - pub fn root_uri(&self) -> String { - match self.integration { - StorageIntegration::Amazon => format!("s3://{}", &self.bucket), - StorageIntegration::Microsoft => format!("az://{}", &self.bucket), - StorageIntegration::Onelake => format!("https://{}", &self.bucket), - StorageIntegration::OnelakeAbfs => format!("abfss://{}", &self.bucket), - StorageIntegration::Google => format!("gs://{}", &self.bucket), - StorageIntegration::Local => format!("file://{}", &self.bucket), - StorageIntegration::Hdfs => format!("hdfs://localhost:9000/{}", &self.bucket), - } - } - - pub fn table_builder(&self, table: TestTables) -> DeltaTableBuilder { - let name = table.as_name(); - let table_uri = format!("{}/{}", self.root_uri(), &name); - DeltaTableBuilder::from_uri(table_uri).with_allow_http(true) - } - - pub fn uri_for_table(&self, table: TestTables) -> String { - format!("{}/{}", self.root_uri(), table.as_name()) - } - - pub async fn load_table(&self, table: TestTables) -> TestResult { - let name = table.as_name(); - self.load_table_with_name(table, name).await - } - - pub async fn load_table_with_name( - &self, - table: TestTables, - name: impl AsRef, - ) -> TestResult { - match self.integration { - StorageIntegration::Local => { - let mut options = CopyOptions::new(); - options.content_only = true; - let dest_path = self.tmp_dir.path().join(name.as_ref()); - std::fs::create_dir_all(&dest_path)?; - copy(table.as_path(), &dest_path, &options)?; - } - StorageIntegration::Amazon => { - let dest = format!("{}/{}", self.root_uri(), name.as_ref()); - s3_cli::copy_directory(table.as_path(), dest)?; - } - StorageIntegration::Microsoft => { - let dest = format!("{}/{}", self.bucket, name.as_ref()); - az_cli::copy_directory(table.as_path(), dest)?; - } - _ => { - let from = table.as_path().as_str().to_owned(); - let to = format!("{}/{}", self.root_uri(), name.as_ref()); - copy_table(from, None, to, None, true).await?; - } - }; - Ok(()) - } -} - -impl Drop for IntegrationContext { - fn drop(&mut self) { - match self.integration { - StorageIntegration::Amazon => { - s3_cli::delete_bucket(self.root_uri()).unwrap(); - s3_cli::delete_lock_table().unwrap(); - } - StorageIntegration::Microsoft => { - az_cli::delete_container(&self.bucket).unwrap(); - } - StorageIntegration::Google => { - gs_cli::delete_bucket(&self.bucket).unwrap(); - } - StorageIntegration::Onelake => (), - StorageIntegration::OnelakeAbfs => (), - StorageIntegration::Local => (), - StorageIntegration::Hdfs => { - hdfs_cli::delete_dir(&self.bucket).unwrap(); - } - }; - } -} - -/// Kinds of storage integration -pub enum StorageIntegration { - Amazon, - Microsoft, - Onelake, - Google, - Local, - Hdfs, - OnelakeAbfs, -} - -impl StorageIntegration { - fn prepare_env(&self) { - match self { - Self::Microsoft => az_cli::prepare_env(), - Self::Onelake => onelake_cli::prepare_env(), - Self::Amazon => s3_cli::prepare_env(), - Self::Google => gs_cli::prepare_env(), - Self::OnelakeAbfs => onelake_cli::prepare_env(), - Self::Local => (), - Self::Hdfs => (), - } - } - - fn create_bucket(&self, name: impl AsRef) -> std::io::Result<()> { - match self { - Self::Microsoft => { - az_cli::create_container(name)?; - Ok(()) - } - Self::Onelake => Ok(()), - Self::OnelakeAbfs => Ok(()), - Self::Amazon => { - std::env::set_var( - "DELTA_DYNAMO_TABLE_NAME", - format!("delta_log_it_{}", rand::thread_rng().gen::()), - ); - s3_cli::create_bucket(format!("s3://{}", name.as_ref()))?; - set_env_if_not_set( - "DYNAMO_LOCK_PARTITION_KEY_VALUE", - format!("s3://{}", name.as_ref()), - ); - s3_cli::create_lock_table()?; - Ok(()) - } - Self::Google => { - gs_cli::create_bucket(name)?; - Ok(()) - } - Self::Local => Ok(()), - Self::Hdfs => { - hdfs_cli::create_dir(name)?; - Ok(()) - } - } - } -} - -/// Reference tables from the test data folder -pub enum TestTables { - Simple, - SimpleCommit, - Golden, - Delta0_8_0Partitioned, - Delta0_8_0SpecialPartitioned, - Custom(String), -} - -impl TestTables { - fn as_path(&self) -> String { - // env "CARGO_MANIFEST_DIR" is "the directory containing the manifest of your package", - // set by `cargo run` or `cargo test`, see: - // https://doc.rust-lang.org/cargo/reference/environment-variables.html - let dir = env!("CARGO_MANIFEST_DIR"); - let data_path = std::path::Path::new(dir).join("tests/data"); - match self { - Self::Simple => data_path.join("simple_table").to_str().unwrap().to_owned(), - Self::SimpleCommit => data_path.join("simple_commit").to_str().unwrap().to_owned(), - Self::Golden => data_path - .join("golden/data-reader-array-primitives") - .to_str() - .unwrap() - .to_owned(), - Self::Delta0_8_0Partitioned => data_path - .join("delta-0.8.0-partitioned") - .to_str() - .unwrap() - .to_owned(), - Self::Delta0_8_0SpecialPartitioned => data_path - .join("delta-0.8.0-special-partition") - .to_str() - .unwrap() - .to_owned(), - // the data path for upload does not apply to custom tables. - Self::Custom(_) => todo!(), - } - } - - pub fn as_name(&self) -> String { - match self { - Self::Simple => "simple".into(), - Self::SimpleCommit => "simple_commit".into(), - Self::Golden => "golden".into(), - Self::Delta0_8_0Partitioned => "delta-0.8.0-partitioned".into(), - Self::Delta0_8_0SpecialPartitioned => "delta-0.8.0-special-partition".into(), - Self::Custom(name) => name.to_owned(), - } - } -} - -/// Set environment variable if it is not set -pub fn set_env_if_not_set(key: impl AsRef, value: impl AsRef) { - if std::env::var(key.as_ref()).is_err() { - std::env::set_var(key.as_ref(), value.as_ref()) - }; -} - -//cli for onelake -pub mod onelake_cli { - use super::set_env_if_not_set; - /// prepare_env - pub fn prepare_env() { - let token = "jwt-token"; - set_env_if_not_set("AZURE_STORAGE_USE_EMULATOR", "0"); - set_env_if_not_set("AZURE_STORAGE_ACCOUNT_NAME", "daily-onelake"); - set_env_if_not_set( - "AZURE_STORAGE_CONTAINER_NAME", - "86bc63cf-5086-42e0-b16d-6bc580d1dc87", - ); - set_env_if_not_set("AZURE_STORAGE_TOKEN", token); - } -} - -/// small wrapper around az cli -pub mod az_cli { - use super::set_env_if_not_set; - use std::process::{Command, ExitStatus}; - - /// Create a new bucket - pub fn create_container(container_name: impl AsRef) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "container", - "create", - "-n", - container_name.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } - - /// delete bucket - pub fn delete_container(container_name: impl AsRef) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "container", - "delete", - "-n", - container_name.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } - - /// copy directory - pub fn copy_directory( - source: impl AsRef, - destination: impl AsRef, - ) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "blob", - "upload-batch", - "-s", - source.as_ref(), - "-d", - destination.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } - - /// prepare_env - pub fn prepare_env() { - set_env_if_not_set("AZURE_STORAGE_USE_EMULATOR", "1"); - set_env_if_not_set("AZURE_STORAGE_ACCOUNT_NAME", "devstoreaccount1"); - set_env_if_not_set("AZURE_STORAGE_ACCOUNT_KEY", "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="); - set_env_if_not_set( - "AZURE_STORAGE_CONNECTION_STRING", - "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://localhost:10000/devstoreaccount1;" - ); - } -} - -/// small wrapper around s3 cli -pub mod s3_cli { - use super::set_env_if_not_set; - use crate::table::builder::s3_storage_options; - use std::process::{Command, ExitStatus, Stdio}; - - /// Create a new bucket - pub fn create_bucket(bucket_name: impl AsRef) -> std::io::Result { - let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) - .expect("variable ENDPOINT must be set to connect to S3"); - let region = std::env::var(s3_storage_options::AWS_REGION) - .expect("variable AWS_REGION must be set to connect to S3"); - let mut child = Command::new("aws") - .args([ - "s3", - "mb", - bucket_name.as_ref(), - "--endpoint-url", - &endpoint, - "--region", - ®ion, - ]) - .spawn() - .expect("aws command is installed"); - child.wait() - } - - /// delete bucket - pub fn delete_bucket(bucket_name: impl AsRef) -> std::io::Result { - let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) - .expect("variable ENDPOINT must be set to connect to S3"); - let mut child = Command::new("aws") - .args([ - "s3", - "rb", - bucket_name.as_ref(), - "--endpoint-url", - &endpoint, - "--force", - ]) - .spawn() - .expect("aws command is installed"); - child.wait() - } - - /// copy directory - pub fn copy_directory( - source: impl AsRef, - destination: impl AsRef, - ) -> std::io::Result { - let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) - .expect("variable ENDPOINT must be set to connect to S3"); - let mut child = Command::new("aws") - .args([ - "s3", - "cp", - source.as_ref(), - destination.as_ref(), - "--endpoint-url", - &endpoint, - "--recursive", - ]) - .spawn() - .expect("aws command is installed"); - child.wait() - } - - /// prepare_env - pub fn prepare_env() { - set_env_if_not_set( - s3_storage_options::AWS_ENDPOINT_URL, - "http://localhost:4566", - ); - set_env_if_not_set(s3_storage_options::AWS_ACCESS_KEY_ID, "deltalake"); - set_env_if_not_set(s3_storage_options::AWS_SECRET_ACCESS_KEY, "weloverust"); - set_env_if_not_set("AWS_DEFAULT_REGION", "us-east-1"); - set_env_if_not_set(s3_storage_options::AWS_REGION, "us-east-1"); - set_env_if_not_set(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); - set_env_if_not_set("DYNAMO_LOCK_TABLE_NAME", "test_table"); - set_env_if_not_set("DYNAMO_LOCK_REFRESH_PERIOD_MILLIS", "100"); - set_env_if_not_set("DYNAMO_LOCK_ADDITIONAL_TIME_TO_WAIT_MILLIS", "100"); - } - - fn create_dynamodb_table( - table_name: &str, - endpoint_url: &str, - attr_definitions: &[&str], - key_schema: &[&str], - ) -> std::io::Result { - println!("creating table {}", table_name); - let args01 = [ - "dynamodb", - "create-table", - "--table-name", - &table_name, - "--endpoint-url", - &endpoint_url, - "--provisioned-throughput", - "ReadCapacityUnits=10,WriteCapacityUnits=10", - "--attribute-definitions", - ]; - let args: Vec<_> = args01 - .iter() - .chain(attr_definitions.iter()) - .chain(["--key-schema"].iter()) - .chain(key_schema) - .collect(); - let mut child = Command::new("aws") - .args(args) - .stdout(Stdio::null()) - .spawn() - .expect("aws command is installed"); - child.wait() - } - - pub fn create_lock_table() -> std::io::Result { - let endpoint_url = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) - .expect("variable AWS_ENDPOINT_URL must be set to connect to S3 emulator"); - let table_name = - std::env::var("DELTA_DYNAMO_TABLE_NAME").unwrap_or_else(|_| "delta_log".into()); - create_dynamodb_table( - &table_name, - &endpoint_url, - &[ - "AttributeName=tablePath,AttributeType=S", - "AttributeName=fileName,AttributeType=S", - ], - &[ - "AttributeName=tablePath,KeyType=HASH", - "AttributeName=fileName,KeyType=RANGE", - ], - ) - } - - fn delete_dynamodb_table(table_name: &str, endpoint_url: &str) -> std::io::Result { - let mut child = Command::new("aws") - .args([ - "dynamodb", - "delete-table", - "--table-name", - &table_name, - "--endpoint-url", - &endpoint_url, - ]) - .stdout(Stdio::null()) - .spawn() - .expect("aws command is installed"); - child.wait() - } - - pub fn delete_lock_table() -> std::io::Result { - let endpoint_url = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) - .expect("variable AWS_ENDPOINT_URL must be set to connect to S3 emulator"); - let table_name = - std::env::var("DELTA_DYNAMO_TABLE_NAME").unwrap_or_else(|_| "delta_log".into()); - delete_dynamodb_table(&table_name, &endpoint_url) - } -} - -/// small wrapper around google api -pub mod gs_cli { - use super::set_env_if_not_set; - use serde_json::json; - use std::process::{Command, ExitStatus}; - - pub fn create_bucket(container_name: impl AsRef) -> std::io::Result { - let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") - .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); - let payload = json!({ "name": container_name.as_ref() }); - let mut child = Command::new("curl") - .args([ - "--insecure", - "-v", - "-X", - "POST", - "--data-binary", - &serde_json::to_string(&payload)?, - "-H", - "Content-Type: application/json", - &endpoint, - ]) - .spawn() - .expect("curl command is installed"); - child.wait() - } - - pub fn delete_bucket(container_name: impl AsRef) -> std::io::Result { - let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") - .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); - let payload = json!({ "name": container_name.as_ref() }); - let mut child = Command::new("curl") - .args([ - "--insecure", - "-v", - "-X", - "DELETE", - "--data-binary", - &serde_json::to_string(&payload)?, - "-H", - "Content-Type: application/json", - &endpoint, - ]) - .spawn() - .expect("curl command is installed"); - child.wait() - } - - /// prepare_env - pub fn prepare_env() { - set_env_if_not_set("GOOGLE_BASE_URL", "http://localhost:4443"); - set_env_if_not_set("GOOGLE_ENDPOINT_URL", "http://localhost:4443/storage/v1/b"); - } -} - -/// small wrapper around hdfs cli -pub mod hdfs_cli { - use std::env; - use std::path::PathBuf; - use std::process::{Command, ExitStatus}; - - fn hdfs_cli_path() -> PathBuf { - let hadoop_home = - env::var("HADOOP_HOME").expect("HADOOP_HOME environment variable not set"); - PathBuf::from(hadoop_home).join("bin").join("hdfs") - } - - pub fn create_dir(dir_name: impl AsRef) -> std::io::Result { - let path = hdfs_cli_path(); - let mut child = Command::new(path) - .args([ - "dfs", - "-mkdir", - "-p", - format!("/{}", dir_name.as_ref()).as_str(), - ]) - .spawn() - .expect("hdfs command is installed"); - child.wait() - } - - pub fn delete_dir(dir_name: impl AsRef) -> std::io::Result { - let path = hdfs_cli_path(); - let mut child = Command::new(path) - .args([ - "dfs", - "-rm", - "-r", - "-f", - format!("/{}", dir_name.as_ref()).as_str(), - ]) - .spawn() - .expect("hdfs command is installed"); - child.wait() - } -} diff --git a/crates/deltalake-core/src/writer/utils.rs b/crates/deltalake-core/src/writer/utils.rs deleted file mode 100644 index 173340f368..0000000000 --- a/crates/deltalake-core/src/writer/utils.rs +++ /dev/null @@ -1,461 +0,0 @@ -//! Handle JSON messages when writing to delta tables -use std::collections::HashMap; -use std::fmt::Display; -use std::io::Write; -use std::sync::Arc; - -use arrow::array::{ - as_boolean_array, as_generic_binary_array, as_largestring_array, as_primitive_array, - as_string_array, Array, -}; -use arrow::datatypes::{ - DataType, Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, - Int8Type, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, TimeUnit, - TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, - TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, -}; -use arrow::json::ReaderBuilder; -use arrow::record_batch::*; -use object_store::path::Path; -use object_store::path::DELIMITER_BYTE; -use parking_lot::RwLock; -use parquet::basic::Compression; -use parquet::file::properties::WriterProperties; -use parquet::schema::types::ColumnPath; -use percent_encoding::{percent_encode, AsciiSet, CONTROLS}; -use serde_json::Value; -use uuid::Uuid; - -use crate::errors::DeltaResult; -use crate::writer::DeltaWriterError; -use crate::NULL_PARTITION_VALUE_DATA_PATH; - -const PARTITION_DATE_FORMAT: &str = "%Y-%m-%d"; -const PARTITION_DATETIME_FORMAT: &str = "%Y-%m-%d %H:%M:%S"; - -#[derive(Debug, Clone, Hash, PartialEq, Eq)] -pub(crate) struct PartitionPath { - path: String, -} - -impl PartitionPath { - pub fn from_hashmap( - partition_columns: &[String], - partition_values: &HashMap>, - ) -> Result { - let mut path_parts = vec![]; - for k in partition_columns.iter() { - let partition_value = partition_values - .get(k) - .ok_or_else(|| DeltaWriterError::MissingPartitionColumn(k.to_string()))?; - let path_part = if let Some(val) = partition_value.as_deref() { - let encoded = percent_encode(val.as_bytes(), INVALID).to_string(); - format!("{k}={encoded}") - } else { - format!("{k}={NULL_PARTITION_VALUE_DATA_PATH}") - }; - path_parts.push(path_part); - } - - Ok(PartitionPath { - path: path_parts.join("/"), - }) - } -} - -const INVALID: &AsciiSet = &CONTROLS - // everything object store needs encoded ... - .add(DELIMITER_BYTE) - .add(b'\\') - .add(b'{') - .add(b'^') - .add(b'}') - .add(b'%') - .add(b'`') - .add(b']') - .add(b'"') - .add(b'>') - .add(b'[') - .add(b'~') - .add(b'<') - .add(b'#') - .add(b'|') - .add(b'\r') - .add(b'\n') - .add(b'*') - .add(b'?') - //... and some more chars illegal on windows - .add(b':'); - -impl From for String { - fn from(path: PartitionPath) -> String { - path.path - } -} - -impl AsRef for PartitionPath { - fn as_ref(&self) -> &str { - &self.path - } -} - -impl Display for PartitionPath { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - self.path.fmt(f) - } -} - -/// Generate the name of the file to be written -/// prefix: The location of the file to be written -/// part_count: Used the indicate that single logical partition was split into multiple physical files -/// starts at 0. Is typically used when writer splits that data due to file size constraints -pub(crate) fn next_data_path( - prefix: &Path, - part_count: usize, - writer_id: &Uuid, - writer_properties: &WriterProperties, -) -> Path { - fn compression_to_str(compression: &Compression) -> &str { - match compression { - // This is to match HADOOP's convention - // https://github.com/apache/parquet-mr/blob/c4977579ab3b149ea045a177b039f055b5408e8f/parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java#L27-L34 - Compression::UNCOMPRESSED => "", - Compression::SNAPPY => ".snappy", - Compression::GZIP(_) => ".gz", - Compression::LZO => ".lzo", - Compression::BROTLI(_) => ".br", - Compression::LZ4 => ".lz4", - Compression::ZSTD(_) => ".zstd", - Compression::LZ4_RAW => ".lz4raw", - } - } - - // We can not access the default column properties but the current implementation will return - // the default compression when the column is not found - let column_path = ColumnPath::new(Vec::new()); - let compression = writer_properties.compression(&column_path); - - let part = format!("{:0>5}", part_count); - - // TODO: what does c000 mean? - let file_name = format!( - "part-{}-{}-c000{}.parquet", - part, - writer_id, - compression_to_str(&compression) - ); - prefix.child(file_name) -} - -/// Convert a vector of json values to a RecordBatch -pub fn record_batch_from_message( - arrow_schema: Arc, - json: &[Value], -) -> DeltaResult { - let mut decoder = ReaderBuilder::new(arrow_schema).build_decoder().unwrap(); - decoder.serialize(json)?; - decoder - .flush()? - .ok_or_else(|| DeltaWriterError::EmptyRecordBatch.into()) -} - -// very naive implementation for plucking the partition value from the first element of a column array. -// ideally, we would do some validation to ensure the record batch containing the passed partition column contains only distinct values. -// if we calculate stats _first_, we can avoid the extra iteration by ensuring max and min match for the column. -// however, stats are optional and can be added later with `dataChange` false log entries, and it may be more appropriate to add stats _later_ to speed up the initial write. -// a happy middle-road might be to compute stats for partition columns only on the initial write since we should validate partition values anyway, and compute additional stats later (at checkpoint time perhaps?). -// also this does not currently support nested partition columns and many other data types. -// TODO is this comment still valid, since we should be sure now, that the arrays where this -// gets applied have a single unique value -pub(crate) fn stringified_partition_value( - arr: &Arc, -) -> Result, DeltaWriterError> { - let data_type = arr.data_type(); - - if arr.is_null(0) { - return Ok(None); - } - - let s = match data_type { - DataType::Int8 => as_primitive_array::(arr).value(0).to_string(), - DataType::Int16 => as_primitive_array::(arr).value(0).to_string(), - DataType::Int32 => as_primitive_array::(arr).value(0).to_string(), - DataType::Int64 => as_primitive_array::(arr).value(0).to_string(), - DataType::UInt8 => as_primitive_array::(arr).value(0).to_string(), - DataType::UInt16 => as_primitive_array::(arr).value(0).to_string(), - DataType::UInt32 => as_primitive_array::(arr).value(0).to_string(), - DataType::UInt64 => as_primitive_array::(arr).value(0).to_string(), - DataType::Float32 => as_primitive_array::(arr).value(0).to_string(), - DataType::Float64 => as_primitive_array::(arr).value(0).to_string(), - DataType::Utf8 => as_string_array(arr).value(0).to_string(), - DataType::LargeUtf8 => as_largestring_array(arr).value(0).to_string(), - DataType::Boolean => as_boolean_array(arr).value(0).to_string(), - DataType::Date32 => as_primitive_array::(arr) - .value_as_date(0) - .unwrap() - .format(PARTITION_DATE_FORMAT) - .to_string(), - DataType::Date64 => as_primitive_array::(arr) - .value_as_date(0) - .unwrap() - .format(PARTITION_DATE_FORMAT) - .to_string(), - DataType::Timestamp(TimeUnit::Second, _) => as_primitive_array::(arr) - .value_as_datetime(0) - .unwrap() - .format(PARTITION_DATETIME_FORMAT) - .to_string(), - DataType::Timestamp(TimeUnit::Millisecond, _) => { - as_primitive_array::(arr) - .value_as_datetime(0) - .unwrap() - .format(PARTITION_DATETIME_FORMAT) - .to_string() - } - DataType::Timestamp(TimeUnit::Microsecond, _) => { - as_primitive_array::(arr) - .value_as_datetime(0) - .unwrap() - .format(PARTITION_DATETIME_FORMAT) - .to_string() - } - DataType::Timestamp(TimeUnit::Nanosecond, _) => { - as_primitive_array::(arr) - .value_as_datetime(0) - .unwrap() - .format(PARTITION_DATETIME_FORMAT) - .to_string() - } - DataType::Binary => as_generic_binary_array::(arr) - .value(0) - .escape_ascii() - .to_string(), - DataType::LargeBinary => as_generic_binary_array::(arr) - .value(0) - .escape_ascii() - .to_string(), - // TODO: handle more types - _ => { - unimplemented!("Unimplemented data type: {:?}", data_type); - } - }; - - Ok(Some(s)) -} - -/// Remove any partition related columns from the record batch -pub(crate) fn record_batch_without_partitions( - record_batch: &RecordBatch, - partition_columns: &[String], -) -> Result { - let mut non_partition_columns = Vec::new(); - for (i, field) in record_batch.schema().fields().iter().enumerate() { - if !partition_columns.contains(field.name()) { - non_partition_columns.push(i); - } - } - - Ok(record_batch.project(&non_partition_columns)?) -} - -/// Arrow schema for the physical file which has partition columns removed -pub(crate) fn arrow_schema_without_partitions( - arrow_schema: &Arc, - partition_columns: &[String], -) -> ArrowSchemaRef { - Arc::new(ArrowSchema::new( - arrow_schema - .fields() - .iter() - .filter(|f| !partition_columns.contains(f.name())) - .map(|f| f.to_owned()) - .collect::>(), - )) -} - -/// An in memory buffer that allows for shared ownership and interior mutability. -/// The underlying buffer is wrapped in an `Arc` and `RwLock`, so cloning the instance -/// allows multiple owners to have access to the same underlying buffer. -#[derive(Debug, Default, Clone)] -pub struct ShareableBuffer { - buffer: Arc>>, -} - -impl ShareableBuffer { - /// Consumes this instance and returns the underlying buffer. - /// Returns None if there are other references to the instance. - pub fn into_inner(self) -> Option> { - Arc::try_unwrap(self.buffer) - .ok() - .map(|lock| lock.into_inner()) - } - - /// Returns a clone of the the underlying buffer as a `Vec`. - pub fn to_vec(&self) -> Vec { - let inner = self.buffer.read(); - (*inner).to_vec() - } - - /// Returns the number of bytes in the underlying buffer. - pub fn len(&self) -> usize { - let inner = self.buffer.read(); - (*inner).len() - } - - /// Returns true if the underlying buffer is empty. - pub fn is_empty(&self) -> bool { - let inner = self.buffer.read(); - (*inner).is_empty() - } - - /// Creates a new instance with buffer initialized from the underylying bytes. - pub fn from_bytes(bytes: &[u8]) -> Self { - Self { - buffer: Arc::new(RwLock::new(bytes.to_vec())), - } - } -} - -impl Write for ShareableBuffer { - fn write(&mut self, buf: &[u8]) -> std::io::Result { - let mut inner = self.buffer.write(); - (*inner).write(buf) - } - - fn flush(&mut self) -> std::io::Result<()> { - let mut inner = self.buffer.write(); - (*inner).flush() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::{ - BinaryArray, BooleanArray, Date32Array, Date64Array, Int16Array, Int32Array, Int64Array, - Int8Array, LargeBinaryArray, StringArray, TimestampMicrosecondArray, - TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, UInt16Array, - UInt32Array, UInt64Array, UInt8Array, - }; - use parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}; - - #[test] - fn test_stringified_partition_value() { - let reference_pairs: Vec<(Arc, Option<&str>)> = vec![ - (Arc::new(Int8Array::from(vec![None])), None), - (Arc::new(Int8Array::from(vec![1])), Some("1")), - (Arc::new(Int16Array::from(vec![1])), Some("1")), - (Arc::new(Int32Array::from(vec![1])), Some("1")), - (Arc::new(Int64Array::from(vec![1])), Some("1")), - (Arc::new(UInt8Array::from(vec![1])), Some("1")), - (Arc::new(UInt16Array::from(vec![1])), Some("1")), - (Arc::new(UInt32Array::from(vec![1])), Some("1")), - (Arc::new(UInt64Array::from(vec![1])), Some("1")), - (Arc::new(UInt8Array::from(vec![1])), Some("1")), - (Arc::new(StringArray::from(vec!["1"])), Some("1")), - (Arc::new(BooleanArray::from(vec![true])), Some("true")), - (Arc::new(BooleanArray::from(vec![false])), Some("false")), - (Arc::new(Date32Array::from(vec![1])), Some("1970-01-02")), - ( - Arc::new(Date64Array::from(vec![86400000])), - Some("1970-01-02"), - ), - ( - Arc::new(TimestampSecondArray::from(vec![1])), - Some("1970-01-01 00:00:01"), - ), - ( - Arc::new(TimestampMillisecondArray::from(vec![1000])), - Some("1970-01-01 00:00:01"), - ), - ( - Arc::new(TimestampMicrosecondArray::from(vec![1000000])), - Some("1970-01-01 00:00:01"), - ), - ( - Arc::new(TimestampNanosecondArray::from(vec![1000000000])), - Some("1970-01-01 00:00:01"), - ), - (Arc::new(BinaryArray::from_vec(vec![b"1"])), Some("1")), - ( - Arc::new(BinaryArray::from_vec(vec![b"\x00\\"])), - Some("\\x00\\\\"), - ), - (Arc::new(LargeBinaryArray::from_vec(vec![b"1"])), Some("1")), - ( - Arc::new(LargeBinaryArray::from_vec(vec![b"\x00\\"])), - Some("\\x00\\\\"), - ), - ]; - for (vals, result) in reference_pairs { - assert_eq!( - stringified_partition_value(&vals).unwrap().as_deref(), - result - ) - } - } - - #[test] - fn test_data_path() { - let prefix = Path::parse("x=0/y=0").unwrap(); - let uuid = Uuid::parse_str("02f09a3f-1624-3b1d-8409-44eff7708208").unwrap(); - - // Validated against Spark - let props = WriterProperties::builder() - .set_compression(Compression::UNCOMPRESSED) - .build(); - - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.parquet" - ); - - let props = WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .build(); - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.snappy.parquet" - ); - - let props = WriterProperties::builder() - .set_compression(Compression::GZIP(GzipLevel::default())) - .build(); - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.gz.parquet" - ); - - let props = WriterProperties::builder() - .set_compression(Compression::LZ4) - .build(); - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.lz4.parquet" - ); - - let props = WriterProperties::builder() - .set_compression(Compression::ZSTD(ZstdLevel::default())) - .build(); - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.zstd.parquet" - ); - - let props = WriterProperties::builder() - .set_compression(Compression::LZ4_RAW) - .build(); - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.lz4raw.parquet" - ); - - let props = WriterProperties::builder() - .set_compression(Compression::BROTLI(BrotliLevel::default())) - .build(); - assert_eq!( - next_data_path(&prefix, 1, &uuid, &props).as_ref(), - "x=0/y=0/part-00001-02f09a3f-1624-3b1d-8409-44eff7708208-c000.br.parquet" - ); - } -} diff --git a/crates/deltalake-core/tests/common/adls.rs b/crates/deltalake-core/tests/common/adls.rs deleted file mode 100644 index 4c441e0325..0000000000 --- a/crates/deltalake-core/tests/common/adls.rs +++ /dev/null @@ -1,93 +0,0 @@ -use super::TestContext; -use chrono::Utc; -use rand::Rng; -use std::collections::HashMap; - -pub struct AzureGen2 { - #[allow(dead_code)] - account_name: String, - #[allow(dead_code)] - account_key: String, - file_system_name: String, -} - -impl Drop for AzureGen2 { - fn drop(&mut self) { - let file_system_name = self.file_system_name.clone(); - az_cli::delete_container(file_system_name).unwrap(); - } -} - -pub async fn setup_azure_gen2_context() -> TestContext { - let mut config = HashMap::new(); - - let storage_account_name = std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let storage_account_key = std::env::var("AZURE_STORAGE_ACCOUNT_KEY").unwrap(); - let storage_container_name = - std::env::var("AZURE_STORAGE_CONTAINER_NAME").unwrap_or_else(|_| "deltars".to_string()); - - let rand: u16 = rand::thread_rng().gen(); - let file_system_name = format!("delta-rs-test-{}-{}", Utc::now().timestamp(), rand); - - az_cli::create_container(&file_system_name).unwrap(); - - let table_uri = format!("azure://{file_system_name}/"); - - config.insert("URI".to_string(), table_uri); - config.insert( - "AZURE_STORAGE_ACCOUNT_NAME".to_string(), - storage_account_name.clone(), - ); - config.insert( - "AZURE_STORAGE_ACCOUNT_KEY".to_string(), - storage_account_key.clone(), - ); - config.insert( - "AZURE_STORAGE_CONTAINER_NAME".to_string(), - storage_container_name, - ); - - TestContext { - storage_context: Some(Box::new(AzureGen2 { - account_name: storage_account_name, - account_key: storage_account_key, - file_system_name, - })), - config, - ..TestContext::default() - } -} - -pub mod az_cli { - use std::process::{Command, ExitStatus}; - - /// Create a new bucket - pub fn create_container(container_name: impl AsRef) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "container", - "create", - "-n", - container_name.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } - - /// delete bucket - pub fn delete_container(container_name: impl AsRef) -> std::io::Result { - let mut child = Command::new("az") - .args([ - "storage", - "container", - "delete", - "-n", - container_name.as_ref(), - ]) - .spawn() - .expect("az command is installed"); - child.wait() - } -} diff --git a/crates/deltalake-core/tests/common/hdfs.rs b/crates/deltalake-core/tests/common/hdfs.rs deleted file mode 100644 index 8da5ef83b6..0000000000 --- a/crates/deltalake-core/tests/common/hdfs.rs +++ /dev/null @@ -1,20 +0,0 @@ -use super::TestContext; -use std::collections::HashMap; - -pub struct Hdfs { - name_node: String, -} - -pub fn setup_hdfs_context() -> TestContext { - let mut config = HashMap::new(); - - let name_node = "hdfs://localhost:9000".to_owned(); - - config.insert("URI".to_owned(), name_node.clone()); - - TestContext { - storage_context: Some(Box::new(Hdfs { name_node })), - config, - ..TestContext::default() - } -} diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc deleted file mode 100644 index c29a858708..0000000000 Binary files a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc deleted file mode 100644 index ccbde4e02b..0000000000 Binary files a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet.crc deleted file mode 100644 index d67ea8a3c2..0000000000 Binary files a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet.crc deleted file mode 100644 index 87694ce3ae..0000000000 Binary files a/crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet.crc deleted file mode 100644 index 35d245353a..0000000000 Binary files a/crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/.part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.8.0/.part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet.crc deleted file mode 100644 index ec945d35b4..0000000000 Binary files a/crates/deltalake-core/tests/data/delta-0.8.0/.part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.crc deleted file mode 100644 index 94a1f7bba2..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":0,"numFiles":0,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.crc deleted file mode 100644 index 36a930c174..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":5489,"numFiles":1,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[5489,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.crc deleted file mode 100644 index 24c62ea4db..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":10978,"numFiles":2,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[10978,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.crc deleted file mode 100644 index a256e617c8..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":16467,"numFiles":3,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[16467,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.crc deleted file mode 100644 index 509df29155..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":21955,"numFiles":4,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[21955,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.crc deleted file mode 100644 index d4c31f771f..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":27444,"numFiles":5,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[27444,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.crc deleted file mode 100644 index 480f26770b..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":32933,"numFiles":6,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[32933,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.crc deleted file mode 100644 index 29c5b3ec22..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":38422,"numFiles":7,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[38422,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.crc deleted file mode 100644 index 6c5f434a3a..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":43911,"numFiles":8,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[43911,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.crc deleted file mode 100644 index 09ea192170..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":49400,"numFiles":9,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[49400,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.crc deleted file mode 100644 index 434a32705e..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":55131,"numFiles":10,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"new_column\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[55131,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.crc deleted file mode 100644 index 9b47d9c1c7..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":60620,"numFiles":11,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"new_column\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[11,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[60620,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.crc b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.crc deleted file mode 100644 index 631a2228d1..0000000000 --- a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.crc +++ /dev/null @@ -1 +0,0 @@ -{"tableSizeBytes":66109,"numFiles":12,"numMetadata":1,"numProtocol":1,"protocol":{"minReaderVersion":1,"minWriterVersion":2},"metadata":{"id":"8d3d2b8a-f091-4d7d-8a37-432a9beaf17b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"integer\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}},{\"name\":\"null\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"decimal\",\"type\":\"decimal(8,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"array\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"nested_struct\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"nested_struct_element\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_of_array_of_map\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"struct_element\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"string\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"new_column\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.checkpoint.writeStatsAsJson":"false","delta.checkpoint.writeStatsAsStruct":"true"},"createdTime":1666652369483},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[66109,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]}} diff --git a/crates/deltalake-core/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc deleted file mode 100644 index c29a858708..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc deleted file mode 100644 index c29a858708..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc deleted file mode 100644 index d67ea8a3c2..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc deleted file mode 100644 index d67ea8a3c2..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc deleted file mode 100644 index 52512aa8c3..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc deleted file mode 100644 index 52512aa8c3..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc deleted file mode 100644 index 52512aa8c3..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc deleted file mode 100644 index 52512aa8c3..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc deleted file mode 100644 index efef74f19f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc deleted file mode 100644 index 3531a2012f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc deleted file mode 100644 index 3c57308100..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc deleted file mode 100644 index 673f94d101..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc deleted file mode 100644 index 6dee6e1233..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc deleted file mode 100644 index d55f40cbc0..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc deleted file mode 100644 index 0cd5190c2c..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc deleted file mode 100644 index 3a4bed33c5..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc deleted file mode 100644 index 42651db57e..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc deleted file mode 100644 index 673f94d101..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc deleted file mode 100644 index aa9bdb761f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc deleted file mode 100644 index 3a0fbe2b5f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc deleted file mode 100644 index 551c8f5742..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc deleted file mode 100644 index 4c990a1391..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc deleted file mode 100644 index 0cd5190c2c..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc deleted file mode 100644 index d55f40cbc0..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc deleted file mode 100644 index 551c8f5742..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc deleted file mode 100644 index cb6d982fbd..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc deleted file mode 100644 index 3531a2012f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc deleted file mode 100644 index 6dee6e1233..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc deleted file mode 100644 index 86c64edb20..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc deleted file mode 100644 index 3a0fbe2b5f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc deleted file mode 100644 index 413fc06a45..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc deleted file mode 100644 index aa9bdb761f..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc deleted file mode 100644 index 47ce4c1985..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc deleted file mode 100644 index 3a4bed33c5..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc deleted file mode 100644 index 4c990a1391..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc deleted file mode 100644 index bcfd8e1eee..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc deleted file mode 100644 index b8335adf96..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc deleted file mode 100644 index 12395f193e..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc deleted file mode 100644 index 22a136f3ec..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_features/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc deleted file mode 100644 index 213b253a30..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_features/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet.crc deleted file mode 100644 index e297c35c57..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet.crc deleted file mode 100644 index 6e1cd84f00..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet.crc deleted file mode 100644 index dd1d93b5d4..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet.crc deleted file mode 100644 index 20323fbaca..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc deleted file mode 100644 index b6346d0159..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet.crc deleted file mode 100644 index d595ff9134..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet.crc deleted file mode 100644 index bd4be2f5ae..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet.crc deleted file mode 100644 index 44cfff06dd..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet.crc deleted file mode 100644 index 4a72f5ae67..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet.crc deleted file mode 100644 index 41ce7af474..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc deleted file mode 100644 index dd1d93b5d4..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc deleted file mode 100644 index fd993bf8ca..0000000000 Binary files a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc and /dev/null differ diff --git a/crates/deltalake-core/tests/integration_object_store.rs b/crates/deltalake-core/tests/integration_object_store.rs deleted file mode 100644 index 31f344aad6..0000000000 --- a/crates/deltalake-core/tests/integration_object_store.rs +++ /dev/null @@ -1,499 +0,0 @@ -#![cfg(feature = "integration_test")] - -use bytes::Bytes; -use deltalake_core::storage::utils::flatten_list_stream; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult}; -use deltalake_core::{DeltaTableBuilder, ObjectStore}; -use object_store::{path::Path, DynObjectStore, Error as ObjectStoreError}; -use serial_test::serial; - -#[tokio::test] -#[serial] -async fn test_object_store_local() -> TestResult { - test_object_store(StorageIntegration::Local, false).await?; - Ok(()) -} - -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_object_store_azure() -> TestResult { - test_object_store(StorageIntegration::Microsoft, false).await?; - Ok(()) -} - -// NOTE: This test is ignored based on [this -// comment](https://github.com/delta-io/delta-rs/pull/1564#issuecomment-1721048753) and we should -// figure out a way to re-enable this test at least in the GitHub Actions CI environment -#[ignore] -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_object_store_onelake() -> TestResult { - let path = Path::from("17d3977c-d46e-4bae-8fed-ff467e674aed/Files/SampleCustomerList.csv"); - read_write_test_onelake(StorageIntegration::Onelake, &path).await?; - Ok(()) -} - -// NOTE: This test is ignored based on [this -// comment](https://github.com/delta-io/delta-rs/pull/1564#issuecomment-1721048753) and we should -// figure out a way to re-enable this test at least in the GitHub Actions CI environment -#[ignore] -#[cfg(feature = "azure")] -#[tokio::test] -#[serial] -async fn test_object_store_onelake_abfs() -> TestResult { - let path = Path::from("17d3977c-d46e-4bae-8fed-ff467e674aed/Files/SampleCustomerList.csv"); - read_write_test_onelake(StorageIntegration::OnelakeAbfs, &path).await?; - Ok(()) -} - -#[cfg(feature = "s3")] -#[tokio::test] -#[ignore = "S3 does not support rename_if_not_exists"] -#[serial] -async fn test_object_store_aws() -> TestResult { - test_object_store(StorageIntegration::Amazon, true).await?; - Ok(()) -} - -// TODO pending emulator support in object store crate -#[ignore] -#[cfg(feature = "gcs")] -#[tokio::test] -#[serial] -async fn test_object_store_google() -> TestResult { - test_object_store(StorageIntegration::Google, false).await?; - Ok(()) -} - -#[cfg(feature = "hdfs")] -#[tokio::test] -#[serial] -async fn test_object_store_hdfs() -> TestResult { - test_object_store(StorageIntegration::Hdfs, false).await?; - Ok(()) -} - -async fn read_write_test_onelake(integration: StorageIntegration, path: &Path) -> TestResult { - let context = IntegrationContext::new(integration)?; - - //println!("line 102-{:#?}",context.root_uri()); - - let delta_store = DeltaTableBuilder::from_uri(&context.root_uri()) - .with_allow_http(true) - .build_storage()? - .object_store(); - - //println!("{:#?}",delta_store); - - let expected = Bytes::from_static(b"test world from delta-rs on friday"); - - delta_store.put(path, expected.clone()).await.unwrap(); - let fetched = delta_store.get(path).await.unwrap().bytes().await.unwrap(); - assert_eq!(expected, fetched); - - for range in [0..10, 3..5, 0..expected.len()] { - let data = delta_store.get_range(path, range.clone()).await.unwrap(); - assert_eq!(&data[..], &expected[range]) - } - - Ok(()) -} - -async fn test_object_store(integration: StorageIntegration, skip_copy: bool) -> TestResult { - let context = IntegrationContext::new(integration)?; - let delta_store = DeltaTableBuilder::from_uri(context.root_uri()) - .with_allow_http(true) - .build_storage()? - .object_store(); - - put_get_delete_list(delta_store.as_ref()).await?; - list_with_delimiter(delta_store.as_ref()).await?; - rename_and_copy(delta_store.as_ref()).await?; - if !skip_copy { - copy_if_not_exists(delta_store.as_ref()).await?; - } - rename_if_not_exists(delta_store.as_ref()).await?; - // get_nonexistent_object(store, None).await?; - Ok(()) -} - -async fn put_get_delete_list(storage: &DynObjectStore) -> TestResult { - delete_fixtures(storage).await?; - - let content_list = flatten_list_stream(storage, None).await?; - assert!( - content_list.is_empty(), - "Expected list to be empty; found: {:?}", - content_list - ); - - let location = Path::from("test_dir/test_file.json"); - - let data = Bytes::from("arbitrary data"); - let expected_data = data.clone(); - storage.put(&location, data).await?; - - let root = Path::from("/"); - - // List everything - let content_list = flatten_list_stream(storage, None).await?; - assert_eq!(content_list, &[location.clone()]); - - // Should behave the same as no prefix - let content_list = flatten_list_stream(storage, Some(&root)).await?; - assert_eq!(content_list, &[location.clone()]); - - // List with delimiter - let result = storage.list_with_delimiter(None).await?; - assert_eq!(&result.objects, &[]); - assert_eq!(result.common_prefixes.len(), 1); - assert_eq!(result.common_prefixes[0], Path::from("test_dir")); - - // Should behave the same as no prefix - let result = storage.list_with_delimiter(Some(&root)).await?; - assert!(result.objects.is_empty()); - assert_eq!(result.common_prefixes.len(), 1); - assert_eq!(result.common_prefixes[0], Path::from("test_dir")); - - // List everything starting with a prefix that should return results - let prefix = Path::from("test_dir"); - let content_list = flatten_list_stream(storage, Some(&prefix)).await?; - assert_eq!(content_list, &[location.clone()]); - - // List everything starting with a prefix that shouldn't return results - let prefix = Path::from("something"); - let content_list = flatten_list_stream(storage, Some(&prefix)).await?; - assert!(content_list.is_empty()); - - let read_data = storage.get(&location).await?.bytes().await?; - assert_eq!(&*read_data, expected_data); - - // Test range request - let range = 3..7; - let range_result = storage.get_range(&location, range.clone()).await; - - let out_of_range = 200..300; - let out_of_range_result = storage.get_range(&location, out_of_range).await; - - let bytes = range_result?; - assert_eq!(bytes, expected_data.slice(range)); - - // Should be a non-fatal error - out_of_range_result.unwrap_err(); - - let ranges = vec![0..1, 2..3, 0..5]; - let bytes = storage.get_ranges(&location, &ranges).await?; - for (range, bytes) in ranges.iter().zip(bytes) { - assert_eq!(bytes, expected_data.slice(range.clone())) - } - - let head = storage.head(&location).await?; - assert_eq!(head.size, expected_data.len()); - - storage.delete(&location).await?; - - let content_list = flatten_list_stream(storage, None).await?; - assert!(content_list.is_empty()); - - let err = storage.get(&location).await.unwrap_err(); - assert!(matches!(err, ObjectStoreError::NotFound { .. }), "{}", err); - - let err = storage.head(&location).await.unwrap_err(); - assert!(matches!(err, ObjectStoreError::NotFound { .. }), "{}", err); - - // Test handling of paths containing an encoded delimiter - - let file_with_delimiter = Path::from_iter(["a", "b/c", "foo.file"]); - storage - .put(&file_with_delimiter, Bytes::from("arbitrary")) - .await?; - - let files = flatten_list_stream(storage, None).await?; - assert_eq!(files, vec![file_with_delimiter.clone()]); - - let files = flatten_list_stream(storage, Some(&Path::from("a/b"))).await?; - assert!(files.is_empty()); - - let files = storage - .list_with_delimiter(Some(&Path::from("a/b"))) - .await?; - assert!(files.common_prefixes.is_empty()); - assert!(files.objects.is_empty()); - - let files = storage.list_with_delimiter(Some(&Path::from("a"))).await?; - assert_eq!(files.common_prefixes, vec![Path::from_iter(["a", "b/c"])]); - assert!(files.objects.is_empty()); - - let files = storage - .list_with_delimiter(Some(&Path::from_iter(["a", "b/c"]))) - .await?; - assert!(files.common_prefixes.is_empty()); - assert_eq!(files.objects.len(), 1); - assert_eq!(files.objects[0].location, file_with_delimiter); - - storage.delete(&file_with_delimiter).await?; - - // Test handling of paths containing non-ASCII characters, e.g. emoji - - let emoji_prefix = Path::from("🙀"); - let emoji_file = Path::from("🙀/😀.parquet"); - storage.put(&emoji_file, Bytes::from("arbitrary")).await?; - - storage.head(&emoji_file).await?; - storage.get(&emoji_file).await?.bytes().await?; - - let files = flatten_list_stream(storage, Some(&emoji_prefix)).await?; - - assert_eq!(files, vec![emoji_file.clone()]); - - let dst = Path::from("foo.parquet"); - storage.copy(&emoji_file, &dst).await?; - let mut files = flatten_list_stream(storage, None).await?; - files.sort_unstable(); - assert_eq!(files, vec![emoji_file.clone(), dst.clone()]); - - storage.delete(&emoji_file).await?; - storage.delete(&dst).await?; - let files = flatten_list_stream(storage, Some(&emoji_prefix)).await?; - assert!(files.is_empty()); - - // Test handling of paths containing percent-encoded sequences - - // "HELLO" percent encoded - let hello_prefix = Path::parse("%48%45%4C%4C%4F")?; - let path = hello_prefix.child("foo.parquet"); - - storage.put(&path, Bytes::from(vec![0, 1])).await?; - let files = flatten_list_stream(storage, Some(&hello_prefix)).await?; - assert_eq!(files, vec![path.clone()]); - - // Cannot list by decoded representation - let files = flatten_list_stream(storage, Some(&Path::from("HELLO"))).await?; - assert!(files.is_empty()); - - // Cannot access by decoded representation - let err = storage - .head(&Path::from("HELLO/foo.parquet")) - .await - .unwrap_err(); - assert!(matches!(err, ObjectStoreError::NotFound { .. }), "{}", err); - - storage.delete(&path).await?; - - // Can also write non-percent encoded sequences - let path = Path::parse("%Q.parquet")?; - storage.put(&path, Bytes::from(vec![0, 1])).await?; - - let files = flatten_list_stream(storage, None).await?; - assert_eq!(files, vec![path.clone()]); - - storage.delete(&path).await?; - Ok(()) -} - -async fn list_with_delimiter(storage: &DynObjectStore) -> TestResult { - delete_fixtures(storage).await?; - - // ==================== check: store is empty ==================== - let content_list = flatten_list_stream(storage, None).await?; - assert!(content_list.is_empty()); - - // ==================== do: create files ==================== - let data = Bytes::from("arbitrary data"); - - let files: Vec<_> = [ - "test_file", - "mydb/wb/000/000/000.segment", - "mydb/wb/000/000/001.segment", - "mydb/wb/000/000/002.segment", - "mydb/wb/001/001/000.segment", - "mydb/wb/foo.json", - "mydb/wbwbwb/111/222/333.segment", - "mydb/data/whatevs", - ] - .iter() - .map(|&s| Path::from(s)) - .collect(); - - for f in &files { - let data = data.clone(); - storage.put(f, data).await?; - } - - // ==================== check: prefix-list `mydb/wb` (directory) ==================== - let prefix = Path::from("mydb/wb"); - - let expected_000 = Path::from("mydb/wb/000"); - let expected_001 = Path::from("mydb/wb/001"); - let expected_location = Path::from("mydb/wb/foo.json"); - - let result = storage.list_with_delimiter(Some(&prefix)).await?; - - assert_eq!(result.common_prefixes, vec![expected_000, expected_001]); - assert_eq!(result.objects.len(), 1); - - let object = &result.objects[0]; - - assert_eq!(object.location, expected_location); - assert_eq!(object.size, data.len()); - - // ==================== check: prefix-list `mydb/wb/000/000/001` (partial filename doesn't match) ==================== - let prefix = Path::from("mydb/wb/000/000/001"); - - let result = storage.list_with_delimiter(Some(&prefix)).await?; - assert!(result.common_prefixes.is_empty()); - assert_eq!(result.objects.len(), 0); - - // ==================== check: prefix-list `not_there` (non-existing prefix) ==================== - let prefix = Path::from("not_there"); - - let result = storage.list_with_delimiter(Some(&prefix)).await?; - assert!(result.common_prefixes.is_empty()); - assert!(result.objects.is_empty()); - - // ==================== do: remove all files ==================== - for f in &files { - storage.delete(f).await?; - } - - // ==================== check: store is empty ==================== - let content_list = flatten_list_stream(storage, None).await?; - assert!(content_list.is_empty()); - Ok(()) -} - -async fn rename_and_copy(storage: &DynObjectStore) -> TestResult { - // Create two objects - let path1 = Path::from("test1"); - let path2 = Path::from("test2"); - let contents1 = Bytes::from("cats"); - let contents2 = Bytes::from("dogs"); - - // copy() make both objects identical - storage.put(&path1, contents1.clone()).await?; - storage.put(&path2, contents2.clone()).await?; - storage.copy(&path1, &path2).await?; - let new_contents = storage.get(&path2).await?.bytes().await?; - assert_eq!(&new_contents, &contents1); - - // rename() copies contents and deletes original - storage.put(&path1, contents1.clone()).await?; - storage.put(&path2, contents2.clone()).await?; - storage.rename(&path1, &path2).await?; - let new_contents = storage.get(&path2).await?.bytes().await?; - assert_eq!(&new_contents, &contents1); - let result = storage.get(&path1).await; - assert!(result.is_err()); - assert!(matches!( - result.unwrap_err(), - ObjectStoreError::NotFound { .. } - )); - - // Clean up - storage.delete(&path2).await?; - Ok(()) -} - -async fn copy_if_not_exists(storage: &DynObjectStore) -> TestResult { - // Create two objects - let path1 = Path::from("test1"); - let path2 = Path::from("test2"); - let contents1 = Bytes::from("cats"); - let contents2 = Bytes::from("dogs"); - - // copy_if_not_exists() errors if destination already exists - storage.put(&path1, contents1.clone()).await?; - storage.put(&path2, contents2.clone()).await?; - let result = storage.copy_if_not_exists(&path1, &path2).await; - assert!(result.is_err()); - assert!(matches!( - result.unwrap_err(), - ObjectStoreError::AlreadyExists { .. } - )); - - // copy_if_not_exists() copies contents and allows deleting original - storage.delete(&path2).await?; - storage.copy_if_not_exists(&path1, &path2).await?; - storage.delete(&path1).await?; - let new_contents = storage.get(&path2).await?.bytes().await?; - assert_eq!(&new_contents, &contents1); - let result = storage.get(&path1).await; - assert!(result.is_err()); - assert!(matches!( - result.unwrap_err(), - ObjectStoreError::NotFound { .. } - )); - - // Clean up - storage.delete(&path2).await?; - Ok(()) -} - -async fn rename_if_not_exists(storage: &DynObjectStore) -> TestResult { - let path1 = Path::from("tmp_file1"); - let path2 = Path::from("tmp_file2"); - storage.put(&path1, bytes::Bytes::new()).await?; - - // delete objects - let result = storage.rename_if_not_exists(&path1, &path2).await; - assert!(result.is_ok()); - assert!(storage.head(&path1).await.is_err()); - assert!(storage.head(&path2).await.is_ok()); - - storage.put(&path1, bytes::Bytes::new()).await?; - let result = storage.rename_if_not_exists(&path1, &path2).await; - assert!(result.is_err()); - assert!(storage.head(&path1).await.is_ok()); - assert!(storage.head(&path2).await.is_ok()); - Ok(()) -} - -// pub(crate) async fn get_nonexistent_object( -// storage: &DynObjectStore, -// location: Option, -// ) -> ObjectStoreResult { -// let location = location.unwrap_or_else(|| Path::from("this_file_should_not_exist")); - -// let err = storage.head(&location).await.unwrap_err(); -// assert!(matches!(err, ObjectStoreError::NotFound { .. })); - -// storage.get(&location).await?.bytes().await -// } - -async fn delete_fixtures(storage: &DynObjectStore) -> TestResult { - let paths = flatten_list_stream(storage, None).await?; - - for f in &paths { - storage.delete(f).await?; - } - Ok(()) -} - -#[tokio::test] -#[serial] -async fn test_object_store_prefixes_local() -> TestResult { - test_object_store_prefixes(StorageIntegration::Local).await?; - Ok(()) -} - -async fn test_object_store_prefixes(integration: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(integration)?; - let prefixes = &["table path", "table path/hello%3F", "你好/😊"]; - for prefix in prefixes { - let rooturi = format!("{}/{}", context.root_uri(), prefix); - let delta_store = DeltaTableBuilder::from_uri(&rooturi) - .with_allow_http(true) - .build_storage()? - .object_store(); - - let contents = Bytes::from("cats"); - let path = Path::from("test"); - delta_store.put(&path, contents.clone()).await?; - let data = delta_store.get(&path).await?.bytes().await?; - assert_eq!(&data, &contents); - } - - Ok(()) -} diff --git a/crates/deltalake-core/tests/integration_read.rs b/crates/deltalake-core/tests/integration_read.rs deleted file mode 100644 index 391020eb59..0000000000 --- a/crates/deltalake-core/tests/integration_read.rs +++ /dev/null @@ -1,339 +0,0 @@ -#![cfg(feature = "integration_test")] - -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; -use deltalake_core::{DeltaTableBuilder, ObjectStore}; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -use maplit::hashmap; -use object_store::path::Path; -use serial_test::serial; - -#[allow(dead_code)] -mod fs_common; - -static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; - -/// TEST_PREFIXES as they should appear in object stores. -static TEST_PREFIXES_ENCODED: &[&str] = &["my%20table", "%E4%BD%A0%E5%A5%BD/%F0%9F%98%8A"]; - -#[cfg(feature = "azure")] -mod azure { - use super::*; - - #[tokio::test] - #[serial] - async fn test_read_tables_azure() -> TestResult { - read_tables(StorageIntegration::Microsoft).await?; - - for (prefix, prefix_encoded) in TEST_PREFIXES.iter().zip(TEST_PREFIXES_ENCODED.iter()) { - read_table_paths(StorageIntegration::Microsoft, prefix, prefix_encoded).await?; - } - - Ok(()) - } -} - -mod local { - use super::*; - - #[tokio::test] - #[serial] - async fn test_read_tables_local() -> TestResult { - read_tables(StorageIntegration::Local).await?; - - for prefix in TEST_PREFIXES { - read_table_paths(StorageIntegration::Local, prefix, prefix).await?; - } - - Ok(()) - } - - #[tokio::test] - async fn test_action_reconciliation() { - let path = "./tests/data/action_reconciliation"; - let mut table = fs_common::create_table(path, None).await; - - // Add a file. - let a = fs_common::add(3 * 60 * 1000); - assert_eq!(1, fs_common::commit_add(&mut table, &a).await); - assert_eq!(table.get_files(), vec![Path::from(a.path.clone())]); - - // Remove added file. - let r = deltalake_core::kernel::Remove { - path: a.path.clone(), - deletion_timestamp: Some(chrono::Utc::now().timestamp_millis()), - data_change: false, - extended_file_metadata: None, - partition_values: None, - size: None, - tags: None, - deletion_vector: None, - base_row_id: None, - default_row_commit_version: None, - }; - - assert_eq!(2, fs_common::commit_removes(&mut table, vec![&r]).await); - assert_eq!(table.get_files().len(), 0); - assert_eq!( - table - .get_state() - .all_tombstones() - .iter() - .map(|r| r.path.as_str()) - .collect::>(), - vec![a.path.as_str()] - ); - - // Add removed file back. - assert_eq!(3, fs_common::commit_add(&mut table, &a).await); - assert_eq!(table.get_files(), vec![Path::from(a.path)]); - // tombstone is removed. - assert_eq!(table.get_state().all_tombstones().len(), 0); - } -} - -#[cfg(feature = "hdfs")] -mod hdfs { - use super::*; - #[tokio::test] - #[serial] - async fn test_read_tables_hdfs() -> TestResult { - Ok(read_tables(StorageIntegration::Hdfs).await?) - } -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -mod s3 { - use super::*; - #[tokio::test] - #[serial] - async fn test_read_tables_aws() -> TestResult { - read_tables(StorageIntegration::Amazon).await?; - - for (prefix, prefix_encoded) in TEST_PREFIXES.iter().zip(TEST_PREFIXES_ENCODED.iter()) { - read_table_paths(StorageIntegration::Amazon, prefix, prefix_encoded).await?; - } - - Ok(()) - } -} - -async fn read_tables(storage: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(storage)?; - context.load_table(TestTables::Simple).await?; - context.load_table(TestTables::Golden).await?; - context - .load_table(TestTables::Delta0_8_0SpecialPartitioned) - .await?; - - read_simple_table(&context).await?; - read_simple_table_with_version(&context).await?; - read_golden(&context).await?; - - Ok(()) -} - -async fn read_table_paths( - storage: StorageIntegration, - table_root: &str, - upload_path: &str, -) -> TestResult { - let context = IntegrationContext::new(storage)?; - context - .load_table_with_name(TestTables::Delta0_8_0SpecialPartitioned, upload_path) - .await?; - - verify_store(&context, table_root).await?; - - read_encoded_table(&context, table_root).await?; - - Ok(()) -} - -async fn verify_store(integration: &IntegrationContext, root_path: &str) -> TestResult { - let table_uri = format!("{}/{}", integration.root_uri(), root_path); - let storage = DeltaTableBuilder::from_uri(table_uri.clone()) - .with_allow_http(true) - .build_storage()? - .object_store(); - - let files = storage.list_with_delimiter(None).await?; - assert_eq!( - vec![ - Path::parse("_delta_log").unwrap(), - Path::parse("x=A%2FA").unwrap(), - Path::parse("x=B%20B").unwrap(), - ], - files.common_prefixes - ); - - Ok(()) -} - -async fn read_encoded_table(integration: &IntegrationContext, root_path: &str) -> TestResult { - let table_uri = format!("{}/{}", integration.root_uri(), root_path); - - let table = DeltaTableBuilder::from_uri(table_uri) - .with_allow_http(true) - .load() - .await?; - - assert_eq!(table.version(), 0); - assert_eq!(table.get_files().len(), 2); - - Ok(()) -} - -async fn read_simple_table(integration: &IntegrationContext) -> TestResult { - let table_uri = integration.uri_for_table(TestTables::Simple); - // the s3 options don't hurt us for other integrations ... - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - let table = DeltaTableBuilder::from_uri(table_uri) - .with_allow_http(true) - .load() - .await?; - #[cfg(not(any(feature = "s3", feature = "s3-native-tls")))] - let table = DeltaTableBuilder::from_uri(table_uri) - .with_allow_http(true) - .load() - .await?; - - assert_eq!(table.version(), 4); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); - assert_eq!( - table.get_files(), - vec![ - Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), - Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), - Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), - Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), - Path::from("part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"), - ] - ); - let tombstones = table.get_state().all_tombstones(); - assert_eq!(tombstones.len(), 31); - assert!(tombstones.contains(&deltalake_core::kernel::Remove { - path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), - deletion_timestamp: Some(1587968596250), - data_change: true, - extended_file_metadata: None, - deletion_vector: None, - base_row_id: None, - default_row_commit_version: None, - size: None, - partition_values: None, - tags: None, - })); - - Ok(()) -} - -async fn read_simple_table_with_version(integration: &IntegrationContext) -> TestResult { - let table_uri = integration.uri_for_table(TestTables::Simple); - - let table = DeltaTableBuilder::from_uri(table_uri) - .with_allow_http(true) - .with_version(3) - .load() - .await?; - - assert_eq!(table.version(), 3); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); - assert_eq!( - table.get_files(), - vec![ - Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), - Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), - Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), - Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), - Path::from("part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet"), - Path::from("part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet"), - ] - ); - let tombstones = table.get_state().all_tombstones(); - assert_eq!(tombstones.len(), 29); - assert!(tombstones.contains(&deltalake_core::kernel::Remove { - path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), - deletion_timestamp: Some(1587968596250), - data_change: true, - tags: None, - partition_values: None, - base_row_id: None, - default_row_commit_version: None, - size: None, - deletion_vector: None, - extended_file_metadata: None, - })); - - Ok(()) -} - -async fn read_golden(integration: &IntegrationContext) -> TestResult { - let table_uri = integration.uri_for_table(TestTables::Golden); - - let table = DeltaTableBuilder::from_uri(table_uri) - .with_allow_http(true) - .load() - .await - .unwrap(); - - assert_eq!(table.version(), 0); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); - - Ok(()) -} - -// TODO we keep teh gcs test around until we can also integrate with CI -#[cfg(feature = "gcs")] -mod gcs { - use object_store::path::Path; - /* - * The storage account to run this test must be provided by the developer and test are executed locally. - * - * To prepare test execution, create a gcs bucket and upload the contents of ./rust/tests/data/simple_table - * into that bucket. - * - * Set the environment variables used for authentication as outlined in rust/src/storage/gcs/mod.rs - * Also set GCS_DELTA_BUCKET for the created bucket name. - * - * remove the ignore statement below and execute tests via 'cargo test --features gcs' - */ - #[ignore] - #[tokio::test] - async fn test_gcs_simple() { - let bucket = std::env::var("GCS_DELTA_BUCKET").unwrap(); - let table = deltalake_core::open_table(format!("gs://{}/simple_table", bucket).as_str()) - .await - .unwrap(); - assert_eq!(table.version(), 4); - assert_eq!(table.protocol().min_writer_version, 2); - assert_eq!(table.protocol().min_reader_version, 1); - assert_eq!( - table.get_files(), - vec![ - Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), - Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), - Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), - Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), - Path::from("part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"), - ] - ); - let tombstones = table.get_state().all_tombstones(); - assert_eq!(tombstones.len(), 31); - assert!(tombstones.contains(&deltalake_core::kernel::Remove { - path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), - deletion_timestamp: Some(1587968596250), - data_change: true, - extended_file_metadata: None, - deletion_vector: None, - base_row_id: None, - default_row_commit_version: None, - size: None, - partition_values: None, - tags: None, - })); - } -} diff --git a/crates/deltalake/Cargo.toml b/crates/deltalake/Cargo.toml index d20a9e0faf..ba3e26cbf9 100644 --- a/crates/deltalake/Cargo.toml +++ b/crates/deltalake/Cargo.toml @@ -1,45 +1,50 @@ [package] name = "deltalake" -version = "0.17.0" -rust-version = "1.64" -authors = ["Qingping Hou "] -homepage = "https://github.com/delta-io/delta.rs" -license = "Apache-2.0" -keywords = ["deltalake", "delta", "datalake"] -description = "Native Delta Lake implementation in Rust" -documentation = "https://docs.rs/deltalake" -repository = "https://github.com/delta-io/delta.rs" -readme = "../README.md" -edition = "2021" +version = "0.17.1" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[package.metadata.docs.rs] +# We cannot use all_features because TLS features are mutually exclusive. +# We cannot use hdfs feature because it requires Java to be installed. +features = ["azure", "datafusion", "gcs", "hdfs", "json", "mount", "python", "s3", "unity-experimental"] [dependencies] -deltalake-core = { path = "../deltalake-core" } -deltalake-catalog-glue = { path = "../deltalake-catalog-glue", optional = true } +deltalake-core = { version = "0.17.1", path = "../core" } +deltalake-aws = { version = "0.1.0", path = "../aws", default-features = false, optional = true } +deltalake-azure = { version = "0.1.0", path = "../azure", optional = true } +deltalake-gcp = { version = "0.1.0", path = "../gcp", optional = true } +deltalake-catalog-glue = { version = "0.1.0", path = "../catalog-glue", optional = true } +deltalake-mount = { version = "0.1.0", path = "../mount", optional = true } [features] # All of these features are just reflected into the core crate until that # functionality is broken apart -azure = ["deltalake-core/azure"] -arrow = ["deltalake-core/arrow"] -default = ["arrow", "deltalake-core/parquet"] +azure = ["deltalake-azure"] +default = [] datafusion = ["deltalake-core/datafusion"] datafusion-ext = ["datafusion"] -gcs = ["deltalake-core/gcs"] +gcs = ["deltalake-gcp"] glue = ["deltalake-catalog-glue"] -glue-native-tls = ["deltalake-catalog-glue/native-tls"] -hdfs = ["deltalake-core/hdfs"] -# used only for integration testing -integration_test = ["deltalake-core/integration_test"] +hdfs = [] json = ["deltalake-core/json"] +mount = ["deltalake-mount"] python = ["deltalake-core/python"] -s3-native-tls = ["deltalake-core/s3-native-tls"] -s3 = ["deltalake-core/s3"] +s3-native-tls = ["deltalake-aws/native-tls"] +s3 = ["deltalake-aws/rustls"] unity-experimental = ["deltalake-core/unity-experimental"] [dev-dependencies] tokio = { version = "1", features = ["macros", "rt-multi-thread"] } chrono = { workspace = true, default-features = false, features = ["clock"] } -log = "*" +tracing = { workspace = true } [[example]] name = "basic_operations" diff --git a/crates/deltalake/examples/recordbatch-writer.rs b/crates/deltalake/examples/recordbatch-writer.rs index 67aac3b962..874b4a10f7 100644 --- a/crates/deltalake/examples/recordbatch-writer.rs +++ b/crates/deltalake/examples/recordbatch-writer.rs @@ -18,8 +18,8 @@ use deltalake::parquet::{ use deltalake::writer::{DeltaWriter, RecordBatchWriter}; use deltalake::Path; use deltalake::*; -use log::*; use std::sync::Arc; +use tracing::*; /* * The main function gets everything started, but does not contain any meaningful diff --git a/crates/deltalake/src/lib.rs b/crates/deltalake/src/lib.rs index 99de56c1d2..c72a72e8bf 100644 --- a/crates/deltalake/src/lib.rs +++ b/crates/deltalake/src/lib.rs @@ -2,3 +2,12 @@ * The deltalake crate is currently just a meta-package shim for deltalake-core */ pub use deltalake_core::*; + +#[cfg(any(feature = "s3", feature = "s3-native-tls"))] +pub use deltalake_aws as aws; +#[cfg(feature = "azure")] +pub use deltalake_azure as azure; +#[cfg(feature = "gcs")] +pub use deltalake_gcp as gcp; +#[cfg(feature = "mount")] +pub use deltalake_mount as mount; diff --git a/crates/gcp/Cargo.toml b/crates/gcp/Cargo.toml new file mode 100644 index 0000000000..daa9042c83 --- /dev/null +++ b/crates/gcp/Cargo.toml @@ -0,0 +1,39 @@ +[package] +name = "deltalake-gcp" +version = "0.1.0" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +deltalake-core = { version = "0.17.0", path = "../core" } +lazy_static = "1" + +# workspace depenndecies +async-trait = { workspace = true } +bytes = { workspace = true } +futures = { workspace = true } +tracing = { workspace = true } +object_store = { workspace = true, features = ["gcp"]} +thiserror = { workspace = true } +tokio = { workspace = true } +regex = { workspace = true } +url = { workspace = true } + +[dev-dependencies] +chrono = { workspace = true } +serial_test = "3" +deltalake-test = { path = "../test" } +pretty_env_logger = "0.5.0" +rand = "0.8" +serde_json = { workspace = true } +tempfile = "3" + +[features] +integration_test = [] diff --git a/crates/gcp/src/config.rs b/crates/gcp/src/config.rs new file mode 100644 index 0000000000..fbc99c7edd --- /dev/null +++ b/crates/gcp/src/config.rs @@ -0,0 +1,163 @@ +//! Auxiliary module for generating a valig Google cloud configuration. +//! +//! Google offers few ways to authenticate against storage accounts and +//! provide credentials for a service principal. Some of this configutaion may +//! partially be specified in the environment. This module establishes a structured +//! way how we discover valid credentials and some heuristics on how they are prioritized. +use std::collections::{hash_map::Entry, HashMap}; +use std::str::FromStr; + +use object_store::gcp::GoogleConfigKey; +use object_store::Error as ObjectStoreError; + +use crate::error::Result; + +lazy_static::lazy_static! { + static ref CREDENTIAL_KEYS: Vec = + Vec::from_iter([ + GoogleConfigKey::ServiceAccountKey, + GoogleConfigKey::ApplicationCredentials, + ]); +} + +/// Credential +enum GcpCredential { + /// Using the service account key + ServiceAccountKey, + /// Using application credentials + ApplicationCredentials, +} + +impl GcpCredential { + /// required configuration keys for variant + fn keys(&self) -> Vec { + match self { + Self::ServiceAccountKey => Vec::from_iter([GoogleConfigKey::ServiceAccountKey]), + Self::ApplicationCredentials => { + Vec::from_iter([GoogleConfigKey::ApplicationCredentials]) + } + } + } +} + +/// Helper struct to create full configuration from passed options and environment +/// +/// Main concern is to pick the desired credential for connecting to starage backend +/// based on a provided configuration and configuration set in the environment. +pub(crate) struct GcpConfigHelper { + config: HashMap, + env_config: HashMap, + priority: Vec, +} + +impl GcpConfigHelper { + /// Create a new [`ConfigHelper`] + pub fn try_new( + config: impl IntoIterator, impl Into)>, + ) -> Result { + let mut env_config = HashMap::new(); + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if key.starts_with("GOOGLE_") { + if let Ok(config_key) = GoogleConfigKey::from_str(&key.to_ascii_lowercase()) { + env_config.insert(config_key, value.to_string()); + } + } + } + } + + Ok(Self { + config: config + .into_iter() + .map(|(key, value)| Ok((GoogleConfigKey::from_str(key.as_ref())?, value.into()))) + .collect::>()?, + env_config, + priority: Vec::from_iter([ + GcpCredential::ServiceAccountKey, + GcpCredential::ApplicationCredentials, + ]), + }) + } + + /// Check if all credential keys are contained in passed config + fn has_full_config(&self, cred: &GcpCredential) -> bool { + cred.keys().iter().all(|key| self.config.contains_key(key)) + } + + /// Check if any credential keys are contained in passed config + fn has_any_config(&self, cred: &GcpCredential) -> bool { + cred.keys().iter().any(|key| self.config.contains_key(key)) + } + + /// Check if all credential keys can be provided using the env + fn has_full_config_with_env(&self, cred: &GcpCredential) -> bool { + cred.keys() + .iter() + .all(|key| self.config.contains_key(key) || self.env_config.contains_key(key)) + } + + /// Generate a cofiguration augmented with options from the environment + pub fn build(mut self) -> Result> { + let mut has_credential = false; + + // try using only passed config options + if !has_credential { + for cred in &self.priority { + if self.has_full_config(cred) { + has_credential = true; + break; + } + } + } + + // try partially avaialbe credentials augmented by environment + if !has_credential { + for cred in &self.priority { + if self.has_any_config(cred) && self.has_full_config_with_env(cred) { + for key in cred.keys() { + if let Entry::Vacant(e) = self.config.entry(key) { + e.insert(self.env_config.get(&key).unwrap().to_owned()); + } + } + has_credential = true; + break; + } + } + } + + // try getting credentials only from the environment + if !has_credential { + for cred in &self.priority { + if self.has_full_config_with_env(cred) { + for key in cred.keys() { + if let Entry::Vacant(e) = self.config.entry(key) { + e.insert(self.env_config.get(&key).unwrap().to_owned()); + } + } + has_credential = true; + break; + } + } + } + + let omit_keys = if has_credential { + CREDENTIAL_KEYS.clone() + } else { + Vec::new() + }; + + // Add keys from the environment to the configuration, as e.g. client configuration options. + // NOTE We have to specifically configure omitting keys, since workload identity can + // work purely using defaults, but partial config may be present in the environment. + // Preference of conflicting configs (e.g. msi resource id vs. client id is handled in object store) + for key in self.env_config.keys() { + if !omit_keys.contains(key) { + if let Entry::Vacant(e) = self.config.entry(*key) { + e.insert(self.env_config.get(key).unwrap().to_owned()); + } + } + } + + Ok(self.config) + } +} diff --git a/crates/gcp/src/error.rs b/crates/gcp/src/error.rs new file mode 100644 index 0000000000..acc18f67f9 --- /dev/null +++ b/crates/gcp/src/error.rs @@ -0,0 +1,22 @@ +use deltalake_core::errors::DeltaTableError; + +pub(crate) type Result = std::result::Result; + +#[derive(thiserror::Error, Debug)] +pub(crate) enum Error { + #[allow(dead_code)] + #[error("failed to parse config: {0}")] + Parse(String), + + #[error(transparent)] + ObjectStore(#[from] object_store::Error), +} + +impl From for DeltaTableError { + fn from(e: Error) -> Self { + match e { + Error::Parse(msg) => DeltaTableError::Generic(msg), + Error::ObjectStore(e) => DeltaTableError::ObjectStore { source: e }, + } + } +} diff --git a/crates/gcp/src/lib.rs b/crates/gcp/src/lib.rs new file mode 100644 index 0000000000..6fe040d398 --- /dev/null +++ b/crates/gcp/src/lib.rs @@ -0,0 +1,68 @@ +use std::collections::HashMap; +use std::str::FromStr; +use std::sync::Arc; + +use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; +use deltalake_core::storage::{ + factories, url_prefix_handler, ObjectStoreFactory, ObjectStoreRef, StorageOptions, +}; +use deltalake_core::{DeltaResult, Path}; +use object_store::gcp::GoogleConfigKey; +use object_store::parse_url_opts; +use url::Url; + +mod config; +pub mod error; + +trait GcpOptions { + fn as_gcp_options(&self) -> HashMap; +} + +impl GcpOptions for StorageOptions { + fn as_gcp_options(&self) -> HashMap { + self.0 + .iter() + .filter_map(|(key, value)| { + Some(( + GoogleConfigKey::from_str(&key.to_ascii_lowercase()).ok()?, + value.clone(), + )) + }) + .collect() + } +} + +#[derive(Clone, Default, Debug)] +pub struct GcpFactory {} + +impl ObjectStoreFactory for GcpFactory { + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + let config = config::GcpConfigHelper::try_new(options.as_gcp_options())?.build()?; + let (store, prefix) = parse_url_opts(url, config)?; + Ok((url_prefix_handler(store, prefix.clone())?, prefix)) + } +} + +impl LogStoreFactory for GcpFactory { + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + Ok(default_logstore(store, location, options)) + } +} + +/// Register an [ObjectStoreFactory] for common Google Cloud [Url] schemes +pub fn register_handlers(_additional_prefixes: Option) { + let factory = Arc::new(GcpFactory {}); + let scheme = &"gs"; + let url = Url::parse(&format!("{}://", scheme)).unwrap(); + factories().insert(url.clone(), factory.clone()); + logstores().insert(url.clone(), factory.clone()); +} diff --git a/crates/gcp/tests/context.rs b/crates/gcp/tests/context.rs new file mode 100644 index 0000000000..b96bd1f41b --- /dev/null +++ b/crates/gcp/tests/context.rs @@ -0,0 +1,159 @@ +use chrono::Utc; +use deltalake_core::errors::DeltaTableError; +use deltalake_core::logstore::LogStore; +use deltalake_core::table::builder::DeltaTableBuilder; +use deltalake_gcp::register_handlers; +use deltalake_test::utils::*; +use futures::StreamExt; +use std::collections::HashMap; +use std::process::ExitStatus; +use std::sync::Arc; +use tempfile::TempDir; + +/// Kinds of storage integration +#[derive(Debug)] +pub struct GcpIntegration { + bucket_name: String, + temp_dir: TempDir, +} + +impl Default for GcpIntegration { + fn default() -> Self { + register_handlers(None); + Self { + bucket_name: format!("test-delta-table-{}", Utc::now().timestamp()), + temp_dir: TempDir::new().unwrap(), + } + } +} + +/// Synchronize the contents of two object stores +pub async fn sync_stores( + from_store: Arc, + to_store: Arc, +) -> Result<(), DeltaTableError> { + let from_store = from_store.object_store().clone(); + let to_store = to_store.object_store().clone(); + // TODO if a table is copied within the same root store (i.e bucket), using copy would be MUCH more efficient + let mut meta_stream = from_store.list(None); + while let Some(file) = meta_stream.next().await { + if let Ok(meta) = file { + let bytes = from_store.get(&meta.location).await?.bytes().await?; + to_store.put(&meta.location, bytes).await?; + } + } + Ok(()) +} + +pub async fn copy_table( + from: impl AsRef, + from_options: Option>, + to: impl AsRef, + to_options: Option>, + allow_http: bool, +) -> Result<(), DeltaTableError> { + let from_store = DeltaTableBuilder::from_uri(from) + .with_storage_options(from_options.unwrap_or_default()) + .with_allow_http(allow_http) + .build_storage()?; + let to_store = DeltaTableBuilder::from_uri(to) + .with_storage_options(to_options.unwrap_or_default()) + .with_allow_http(allow_http) + .build_storage()?; + sync_stores(from_store, to_store).await +} + +impl StorageIntegration for GcpIntegration { + fn prepare_env(&self) { + gs_cli::prepare_env(); + let base_url = std::env::var("GOOGLE_BASE_URL").unwrap(); + let token = serde_json::json!({"gcs_base_url": base_url, "disable_oauth": true, "client_email": "", "private_key": "", "private_key_id": ""}); + let account_path = self.temp_dir.path().join("gcs.json"); + println!("accoutn_path: {account_path:?}"); + std::fs::write(&account_path, serde_json::to_vec(&token).unwrap()).unwrap(); + std::env::set_var( + "GOOGLE_SERVICE_ACCOUNT", + account_path.as_path().to_str().unwrap(), + ); + } + + fn create_bucket(&self) -> std::io::Result { + gs_cli::create_bucket(self.bucket_name()) + } + + fn bucket_name(&self) -> String { + self.bucket_name.clone() + } + + fn root_uri(&self) -> String { + format!("gs://{}", self.bucket_name()) + } + + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + use futures::executor::block_on; + + let to = format!("{}/{}", self.root_uri(), destination); + let _ = block_on(copy_table(source.to_owned(), None, to, None, true)); + Ok(ExitStatus::default()) + } +} + +impl GcpIntegration { + fn delete_bucket(&self) -> std::io::Result { + gs_cli::delete_bucket(self.bucket_name.clone()) + } +} + +/// small wrapper around google api +pub mod gs_cli { + use super::set_env_if_not_set; + use std::process::{Command, ExitStatus}; + + pub fn create_bucket(container_name: impl AsRef) -> std::io::Result { + let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") + .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); + let payload = serde_json::json!({ "name": container_name.as_ref() }); + let mut child = Command::new("curl") + .args([ + "--insecure", + "-v", + "-X", + "POST", + "--data-binary", + &serde_json::to_string(&payload)?, + "-H", + "Content-Type: application/json", + &endpoint, + ]) + .spawn() + .expect("curl command is installed"); + child.wait() + } + + pub fn delete_bucket(container_name: impl AsRef) -> std::io::Result { + let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") + .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); + let payload = serde_json::json!({ "name": container_name.as_ref() }); + let mut child = Command::new("curl") + .args([ + "--insecure", + "-v", + "-X", + "DELETE", + "--data-binary", + &serde_json::to_string(&payload)?, + "-H", + "Content-Type: application/json", + &endpoint, + ]) + .spawn() + .expect("curl command is installed"); + child.wait() + } + + /// prepare_env + pub fn prepare_env() { + set_env_if_not_set("GOOGLE_BASE_URL", "http://localhost:4443"); + set_env_if_not_set("GOOGLE_ENDPOINT_URL", "http://localhost:4443/storage/v1/b"); + } +} diff --git a/crates/gcp/tests/integration.rs b/crates/gcp/tests/integration.rs new file mode 100644 index 0000000000..d52e2bddc9 --- /dev/null +++ b/crates/gcp/tests/integration.rs @@ -0,0 +1,38 @@ +#![cfg(feature = "integration_test")] + +use deltalake_test::read::read_table_paths; +use deltalake_test::{test_concurrent_writes, test_read_tables, IntegrationContext, TestResult}; +use serial_test::serial; + +mod context; +use context::*; + +static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; +/// TEST_PREFIXES as they should appear in object stores. +static TEST_PREFIXES_ENCODED: &[&str] = &["my%20table", "%E4%BD%A0%E5%A5%BD/%F0%9F%98%8A"]; + +#[tokio::test] +#[serial] +#[ignore = "The GCP tests currently hang"] +async fn test_read_tables_gcp() -> TestResult { + let context = IntegrationContext::new(Box::new(GcpIntegration::default()))?; + + test_read_tables(&context).await?; + + for (prefix, prefix_encoded) in TEST_PREFIXES.iter().zip(TEST_PREFIXES_ENCODED.iter()) { + read_table_paths(&context, prefix, prefix_encoded).await?; + } + + Ok(()) +} + +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] +#[serial] +#[ignore = "The GCP tests currently hang"] +async fn test_concurrency_gcp() -> TestResult { + let context = IntegrationContext::new(Box::new(GcpIntegration::default()))?; + + test_concurrent_writes(&context).await?; + + Ok(()) +} diff --git a/crates/mount/Cargo.toml b/crates/mount/Cargo.toml new file mode 100644 index 0000000000..979a19592a --- /dev/null +++ b/crates/mount/Cargo.toml @@ -0,0 +1,43 @@ +[package] +name = "deltalake-mount" +version = "0.1.0" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true + +[dependencies] +deltalake-core = { version = "0.17.0", path = "../core", features = [ + "datafusion", +] } +lazy_static = "1" +errno = "0.3" + +# workspace depenndecies +async-trait = { workspace = true } +bytes = { workspace = true } +futures = { workspace = true } +tracing = { workspace = true } +object_store = { workspace = true } +thiserror = { workspace = true } +tokio = { workspace = true } +regex = { workspace = true } +url = { workspace = true } + +[dev-dependencies] +chrono = { workspace = true } +serial_test = "3" +deltalake-test = { path = "../test" } +pretty_env_logger = "0.5.0" +rand = "0.8" +serde_json = { workspace = true } +tempfile = "3" +fs_extra = "1.3.0" + +[features] +integration_test = [] diff --git a/crates/mount/src/config.rs b/crates/mount/src/config.rs new file mode 100644 index 0000000000..79dbfc88d0 --- /dev/null +++ b/crates/mount/src/config.rs @@ -0,0 +1,80 @@ +//! Auxiliary module for generating a valig Mount configuration. +use std::collections::{hash_map::Entry, HashMap}; +use std::str::FromStr; + +use crate::error::{Error, Result}; + +/// Typed property keys that can be defined on a mounted path +#[derive(PartialEq, Eq, Hash, Clone, Debug, Copy)] +#[non_exhaustive] +pub enum MountConfigKey { + /// If set to "true", allows creating commits without concurrent writer protection. + /// Only safe if there is one writer to a given table. + AllowUnsafeRename, +} + +impl AsRef for MountConfigKey { + fn as_ref(&self) -> &str { + match self { + Self::AllowUnsafeRename => "mount_allow_unsafe_rename", + } + } +} + +impl FromStr for MountConfigKey { + type Err = Error; + + fn from_str(s: &str) -> Result { + match s { + "mount_allow_unsafe_rename" | "allow_unsafe_rename" => Ok(Self::AllowUnsafeRename), + _ => Err(Error::UnknownConfigKey(s.to_string())), + } + } +} + +/// Helper struct to create full configuration from passed options and environment +pub(crate) struct MountConfigHelper { + config: HashMap, + env_config: HashMap, +} + +impl MountConfigHelper { + /// Create a new [`ConfigHelper`] + pub fn try_new( + config: impl IntoIterator, impl Into)>, + ) -> Result { + let mut env_config = HashMap::new(); + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if key.starts_with("MOUNT_") { + if let Ok(config_key) = MountConfigKey::from_str(&key.to_ascii_lowercase()) { + env_config.insert(config_key, value.to_string()); + } + } + } + } + + Ok(Self { + config: config + .into_iter() + .map(|(key, value)| Ok((MountConfigKey::from_str(key.as_ref())?, value.into()))) + .collect::>()?, + env_config, + }) + } + + /// Generate a cofiguration augmented with options from the environment + pub fn build(mut self) -> Result> { + // Add keys from the environment to the configuration, as e.g. client configuration options. + // NOTE We have to specifically configure omitting keys, since workload identity can + // work purely using defaults, but partial config may be present in the environment. + // Preference of conflicting configs (e.g. msi resource id vs. client id is handled in object store) + for key in self.env_config.keys() { + if let Entry::Vacant(e) = self.config.entry(*key) { + e.insert(self.env_config.get(key).unwrap().to_owned()); + } + } + + Ok(self.config) + } +} diff --git a/crates/mount/src/error.rs b/crates/mount/src/error.rs new file mode 100644 index 0000000000..3693b0be07 --- /dev/null +++ b/crates/mount/src/error.rs @@ -0,0 +1,33 @@ +use deltalake_core::errors::DeltaTableError; + +pub(crate) type Result = std::result::Result; + +#[derive(thiserror::Error, Debug)] +pub enum Error { + #[allow(dead_code)] + #[error("failed to parse config: {0}")] + Parse(String), + + /// Unknown configuration key + #[error("Unknown configuration key: {0}")] + UnknownConfigKey(String), + + #[error("The `allow_unsafe_rename` parameter must be specified")] + AllowUnsafeRenameNotSpecified, + + #[error(transparent)] + ObjectStore(#[from] object_store::Error), +} + +impl From for DeltaTableError { + fn from(e: Error) -> Self { + match e { + Error::Parse(msg) => DeltaTableError::Generic(msg), + Error::UnknownConfigKey(msg) => DeltaTableError::Generic(msg), + Error::AllowUnsafeRenameNotSpecified => DeltaTableError::Generic( + "The `allow_unsafe_rename` parameter must be specified".to_string(), + ), + Error::ObjectStore(e) => DeltaTableError::ObjectStore { source: e }, + } + } +} diff --git a/crates/mount/src/file.rs b/crates/mount/src/file.rs new file mode 100644 index 0000000000..02e5f82e6c --- /dev/null +++ b/crates/mount/src/file.rs @@ -0,0 +1,349 @@ +//! Mount file storage backend. This backend read and write objects from mounted filesystem. +//! +//! The mount file storage backend is not multi-writer safe. + +use bytes::Bytes; +use futures::stream::BoxStream; +use object_store::{ + local::LocalFileSystem, path::Path as ObjectStorePath, Error as ObjectStoreError, GetOptions, + GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, PutOptions, PutResult, + Result as ObjectStoreResult, +}; +use std::ops::Range; +use std::sync::Arc; +use tokio::io::AsyncWrite; +use url::Url; + +pub(crate) const STORE_NAME: &str = "MountObjectStore"; + +/// Error raised by storage lock client +#[derive(thiserror::Error, Debug)] +#[allow(dead_code)] +pub enum LocalFileSystemError { + /// Object exists already at path + #[error("Object exists already at path: {} ({:?})", path, source)] + AlreadyExists { + /// Path of the already existing file + path: String, + /// Originating error + source: Box, + }, + + /// Object not found at the given path + #[error("Object not found at path: {} ({:?})", path, source)] + NotFound { + /// Provided path which does not exist + path: String, + /// Originating error + source: Box, + }, + + /// Invalid argument sent to OS call + #[error("Invalid argument in OS call for path: {} ({:?})", path, source)] + InvalidArgument { + /// Provided path + path: String, + /// Originating error + source: errno::Errno, + }, + + /// Null error for path for FFI + #[error("Null error in FFI for path: {} ({:?})", path, source)] + NullError { + /// Given path + path: String, + /// Originating error + source: std::ffi::NulError, + }, + + /// Generic catch-all error for this store + #[error("Generic error in store: {} ({:?})", store, source)] + Generic { + /// String name of the object store + store: &'static str, + /// Originating error + source: Box, + }, + + /// Errors from the Tokio runtime + #[error("Error executing async task for path: {} ({:?})", path, source)] + Tokio { + /// Path + path: String, + /// Originating error + source: tokio::task::JoinError, + }, +} + +impl From for ObjectStoreError { + fn from(e: LocalFileSystemError) -> Self { + match e { + LocalFileSystemError::AlreadyExists { path, source } => { + ObjectStoreError::AlreadyExists { path, source } + } + LocalFileSystemError::NotFound { path, source } => { + ObjectStoreError::NotFound { path, source } + } + LocalFileSystemError::InvalidArgument { source, .. } => ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(source), + }, + LocalFileSystemError::NullError { source, .. } => ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(source), + }, + LocalFileSystemError::Tokio { source, .. } => ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(source), + }, + LocalFileSystemError::Generic { store, source } => { + ObjectStoreError::Generic { store, source } + } + } + } +} + +/// Mount File Storage Backend. +/// Note that it's non-atomic writing and may leave the filesystem in an inconsistent state if it fails. +#[derive(Debug)] +pub struct MountFileStorageBackend { + inner: Arc, + root_url: Arc, +} + +impl MountFileStorageBackend { + /// Creates a new MountFileStorageBackend. + pub fn try_new(path: impl AsRef) -> ObjectStoreResult { + Ok(Self { + root_url: Arc::new(Self::path_to_root_url(path.as_ref())?), + inner: Arc::new(LocalFileSystem::new_with_prefix(path)?), + }) + } + + fn path_to_root_url(path: &std::path::Path) -> ObjectStoreResult { + let root_path = + std::fs::canonicalize(path).map_err(|e| object_store::Error::InvalidPath { + source: object_store::path::Error::Canonicalize { + path: path.into(), + source: e, + }, + })?; + + Url::from_file_path(root_path).map_err(|_| object_store::Error::InvalidPath { + source: object_store::path::Error::InvalidPath { path: path.into() }, + }) + } + + /// Return an absolute filesystem path of the given location + fn path_to_filesystem(&self, location: &ObjectStorePath) -> String { + let mut url = self.root_url.as_ref().clone(); + url.path_segments_mut() + .expect("url path") + // technically not necessary as Path ignores empty segments + // but avoids creating paths with "//" which look odd in error messages. + .pop_if_empty() + .extend(location.parts()); + + url.to_file_path().unwrap().to_str().unwrap().to_owned() + } +} + +impl std::fmt::Display for MountFileStorageBackend { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "MountFileStorageBackend") + } +} + +#[async_trait::async_trait] +impl ObjectStore for MountFileStorageBackend { + async fn put(&self, location: &ObjectStorePath, bytes: Bytes) -> ObjectStoreResult { + self.inner.put(location, bytes).await + } + + async fn put_opts( + &self, + location: &ObjectStorePath, + bytes: Bytes, + options: PutOptions, + ) -> ObjectStoreResult { + self.inner.put_opts(location, bytes, options).await + } + + async fn get(&self, location: &ObjectStorePath) -> ObjectStoreResult { + self.inner.get(location).await + } + + async fn get_opts( + &self, + location: &ObjectStorePath, + options: GetOptions, + ) -> ObjectStoreResult { + self.inner.get_opts(location, options).await + } + + async fn get_range( + &self, + location: &ObjectStorePath, + range: Range, + ) -> ObjectStoreResult { + self.inner.get_range(location, range).await + } + + async fn head(&self, location: &ObjectStorePath) -> ObjectStoreResult { + self.inner.head(location).await + } + + async fn delete(&self, location: &ObjectStorePath) -> ObjectStoreResult<()> { + self.inner.delete(location).await + } + + fn list( + &self, + prefix: Option<&ObjectStorePath>, + ) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list(prefix) + } + + fn list_with_offset( + &self, + prefix: Option<&ObjectStorePath>, + offset: &ObjectStorePath, + ) -> BoxStream<'_, ObjectStoreResult> { + self.inner.list_with_offset(prefix, offset) + } + + async fn list_with_delimiter( + &self, + prefix: Option<&ObjectStorePath>, + ) -> ObjectStoreResult { + self.inner.list_with_delimiter(prefix).await + } + + async fn copy(&self, from: &ObjectStorePath, to: &ObjectStorePath) -> ObjectStoreResult<()> { + self.inner.copy(from, to).await + } + + async fn copy_if_not_exists( + &self, + from: &ObjectStorePath, + to: &ObjectStorePath, + ) -> ObjectStoreResult<()> { + self.inner.copy_if_not_exists(from, to).await + } + + async fn rename_if_not_exists( + &self, + from: &ObjectStorePath, + to: &ObjectStorePath, + ) -> ObjectStoreResult<()> { + let path_from = self.path_to_filesystem(from); + let path_to = self.path_to_filesystem(to); + Ok(regular_rename(path_from.as_ref(), path_to.as_ref()).await?) + } + + async fn put_multipart( + &self, + location: &ObjectStorePath, + ) -> ObjectStoreResult<(MultipartId, Box)> { + self.inner.put_multipart(location).await + } + + async fn abort_multipart( + &self, + location: &ObjectStorePath, + multipart_id: &MultipartId, + ) -> ObjectStoreResult<()> { + self.inner.abort_multipart(location, multipart_id).await + } +} + +/// Regular renames `from` to `to`. +/// `from` has to exist, but `to` is not, otherwise the operation will fail. +/// It's not atomic and cannot be called in parallel with other operations on the same file. +#[inline] +async fn regular_rename(from: &str, to: &str) -> Result<(), LocalFileSystemError> { + let from_path = String::from(from); + let to_path = String::from(to); + + tokio::task::spawn_blocking(move || { + if std::fs::metadata(&to_path).is_ok() { + Err(LocalFileSystemError::AlreadyExists { + path: to_path, + source: Box::new(std::io::Error::new( + std::io::ErrorKind::AlreadyExists, + "Already exists", + )), + }) + } else { + std::fs::rename(&from_path, &to_path).map_err(|err| { + if err.kind() == std::io::ErrorKind::NotFound { + LocalFileSystemError::NotFound { + path: from_path.clone(), + source: Box::new(err), + } + } else { + LocalFileSystemError::Generic { + store: STORE_NAME, + source: Box::new(err), + } + } + })?; + Ok(()) + } + }) + .await + .unwrap() +} + +#[cfg(test)] +mod tests { + use super::*; + use std::fs::File; + use std::io::Write; + use std::path::{Path, PathBuf}; + + #[tokio::test] + async fn test_regular_rename() { + let tmp_dir = tempfile::tempdir().unwrap(); + let a = create_file(tmp_dir.path(), "a"); + let b = create_file(tmp_dir.path(), "b"); + let c = &tmp_dir.path().join("c"); + + // unsuccessful move not_exists to C, not_exists is missing + let result = regular_rename("not_exists", c.to_str().unwrap()).await; + assert!(matches!( + result.expect_err("nonexistent should fail"), + LocalFileSystemError::NotFound { .. } + )); + + // successful move A to C + assert!(a.exists()); + assert!(!c.exists()); + match regular_rename(a.to_str().unwrap(), c.to_str().unwrap()).await { + Err(LocalFileSystemError::InvalidArgument {source, ..}) => + panic!("expected success, got: {source:?}. Note: atomically renaming Windows files from WSL2 is not supported."), + Err(e) => panic!("expected success, got: {e:?}"), + _ => {} + } + assert!(!a.exists()); + assert!(c.exists()); + + // unsuccessful move B to C, C already exists, B is not deleted + assert!(b.exists()); + match regular_rename(b.to_str().unwrap(), c.to_str().unwrap()).await { + Err(LocalFileSystemError::AlreadyExists { path, .. }) => { + assert_eq!(path, c.to_str().unwrap()) + } + _ => panic!("unexpected"), + } + assert!(b.exists()); + assert_eq!(std::fs::read_to_string(c).unwrap(), "a"); + } + + fn create_file(dir: &Path, name: &str) -> PathBuf { + let path = dir.join(name); + let mut file = File::create(&path).unwrap(); + file.write_all(name.as_bytes()).unwrap(); + path + } +} diff --git a/crates/mount/src/lib.rs b/crates/mount/src/lib.rs new file mode 100644 index 0000000000..2decb92b6c --- /dev/null +++ b/crates/mount/src/lib.rs @@ -0,0 +1,102 @@ +use std::collections::HashMap; +use std::str::FromStr; +use std::sync::Arc; + +use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; +use deltalake_core::storage::{ + factories, str_is_truthy, ObjectStoreFactory, ObjectStoreRef, StorageOptions, +}; +use deltalake_core::{DeltaResult, DeltaTableError, Path}; +use object_store::local::LocalFileSystem; +use url::Url; + +mod config; +pub mod error; +mod file; + +trait MountOptions { + fn as_mount_options(&self) -> HashMap; +} + +impl MountOptions for StorageOptions { + fn as_mount_options(&self) -> HashMap { + self.0 + .iter() + .filter_map(|(key, value)| { + Some(( + config::MountConfigKey::from_str(&key.to_ascii_lowercase()).ok()?, + value.clone(), + )) + }) + .collect() + } +} + +#[derive(Clone, Default, Debug)] +pub struct MountFactory {} + +impl ObjectStoreFactory for MountFactory { + fn parse_url_opts( + &self, + url: &Url, + options: &StorageOptions, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + let config = config::MountConfigHelper::try_new(options.as_mount_options())?.build()?; + + let allow_unsafe_rename = str_is_truthy( + config + .get(&config::MountConfigKey::AllowUnsafeRename) + .unwrap_or(&String::new()), + ); + + match url.scheme() { + "dbfs" => { + if !allow_unsafe_rename { + // Just let the user know that they need to set the allow_unsafe_rename option + return Err(error::Error::AllowUnsafeRenameNotSpecified.into()); + } + // We need to convert the dbfs url to a file url + let new_url = Url::parse(&format!("file:///dbfs{}", url.path())).unwrap(); + let store = Arc::new(file::MountFileStorageBackend::try_new( + new_url.to_file_path().unwrap(), + )?) as ObjectStoreRef; + Ok((store, Path::from("/"))) + } + "file" => { + if allow_unsafe_rename { + let store = Arc::new(file::MountFileStorageBackend::try_new( + url.to_file_path().unwrap(), + )?) as ObjectStoreRef; + Ok((store, Path::from("/"))) + } else { + let store = Arc::new(LocalFileSystem::new_with_prefix( + url.to_file_path().unwrap(), + )?) as ObjectStoreRef; + Ok((store, Path::from("/"))) + } + } + _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), + } + } +} + +impl LogStoreFactory for MountFactory { + fn with_options( + &self, + store: ObjectStoreRef, + location: &Url, + options: &StorageOptions, + ) -> DeltaResult> { + Ok(default_logstore(store, location, options)) + } +} + +/// Register an [ObjectStoreFactory] for common Mount [Url] schemes +pub fn register_handlers(_additional_prefixes: Option) { + let factory = Arc::new(MountFactory {}); + for scheme in ["dbfs", "file"].iter() { + let url = Url::parse(&format!("{}://", scheme)).unwrap(); + factories().insert(url.clone(), factory.clone()); + logstores().insert(url.clone(), factory.clone()); + } +} diff --git a/crates/mount/tests/context.rs b/crates/mount/tests/context.rs new file mode 100644 index 0000000000..d7977b36de --- /dev/null +++ b/crates/mount/tests/context.rs @@ -0,0 +1,85 @@ +use deltalake_mount::register_handlers; +use deltalake_test::utils::{set_env_if_not_set, StorageIntegration}; +use fs_extra::dir::{copy, CopyOptions}; +use std::process::ExitStatus; +use tempfile::{tempdir, TempDir}; + +pub struct MountIntegration { + tmp_dir: TempDir, +} + +impl Default for MountIntegration { + fn default() -> Self { + register_handlers(None); + Self { + tmp_dir: tempdir().expect("Failed to make temp dir"), + } + } +} + +impl StorageIntegration for MountIntegration { + fn create_bucket(&self) -> std::io::Result { + Ok(ExitStatus::default()) + } + + fn prepare_env(&self) { + set_env_if_not_set("MOUNT_ALLOW_UNSAFE_RENAME", "true"); + } + fn bucket_name(&self) -> String { + self.tmp_dir.as_ref().to_str().unwrap().to_owned() + } + fn root_uri(&self) -> String { + format!("file://{}", self.bucket_name()) + } + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + let mut options = CopyOptions::new(); + options.content_only = true; + let dest_path = self.tmp_dir.path().join(destination); + std::fs::create_dir_all(&dest_path)?; + copy(source, &dest_path, &options).expect("Failed to copy"); + Ok(ExitStatus::default()) + } +} + +pub struct DbfsIntegration { + tmp_dir: TempDir, +} + +impl Default for DbfsIntegration { + fn default() -> Self { + register_handlers(None); + Self { + tmp_dir: tempdir().expect("Failed to make temp dir"), + } + } +} + +impl StorageIntegration for DbfsIntegration { + fn create_bucket(&self) -> std::io::Result { + Ok(ExitStatus::default()) + } + + fn prepare_env(&self) { + set_env_if_not_set("MOUNT_ALLOW_UNSAFE_RENAME", "true"); + std::fs::create_dir_all(format!("/dbfs{}", self.tmp_dir.as_ref().to_str().unwrap())) + .expect("Failed to create dir"); + } + fn bucket_name(&self) -> String { + self.tmp_dir.as_ref().to_str().unwrap().to_owned() + } + fn root_uri(&self) -> String { + format!("dbfs:{}", self.bucket_name()) + } + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + let mut options = CopyOptions::new(); + options.content_only = true; + let dest_path = format!( + "/dbfs{}/{}", + self.tmp_dir.as_ref().to_str().unwrap(), + destination + ); + std::fs::create_dir_all(&dest_path)?; + copy(source, &dest_path, &options).expect("Failed to copy"); + Ok(ExitStatus::default()) + } +} diff --git a/crates/mount/tests/integration.rs b/crates/mount/tests/integration.rs new file mode 100644 index 0000000000..14fcbcdc95 --- /dev/null +++ b/crates/mount/tests/integration.rs @@ -0,0 +1,39 @@ +#![cfg(feature = "integration_test")] + +use deltalake_test::read::read_table_paths; +use deltalake_test::{test_read_tables, IntegrationContext, TestResult}; +use serial_test::serial; + +mod context; +use context::*; + +static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; + +#[tokio::test] +#[serial] +async fn test_integration_local() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + + test_read_tables(&context).await?; + + for prefix in TEST_PREFIXES { + read_table_paths(&context, prefix, prefix).await?; + } + + Ok(()) +} + +#[tokio::test] +#[serial] +#[ignore = "The DBFS tests currently hang due to CI pipeline cannot write to /dbfs"] +async fn test_integration_dbfs() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + + test_read_tables(&context).await?; + + for prefix in TEST_PREFIXES { + read_table_paths(&context, prefix, prefix).await?; + } + + Ok(()) +} diff --git a/crates/deltalake-sql/Cargo.toml b/crates/sql/Cargo.toml similarity index 58% rename from crates/deltalake-sql/Cargo.toml rename to crates/sql/Cargo.toml index f6fb345a18..14457117b7 100644 --- a/crates/deltalake-sql/Cargo.toml +++ b/crates/sql/Cargo.toml @@ -1,7 +1,15 @@ [package] name = "deltalake-sql" version = "0.1.0" -edition = "2021" +authors.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +description.workspace = true +license.workspace = true +repository.workspace = true +rust-version.workspace = true # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/crates/deltalake-sql/README.md b/crates/sql/README.md similarity index 100% rename from crates/deltalake-sql/README.md rename to crates/sql/README.md diff --git a/crates/deltalake-sql/src/lib.rs b/crates/sql/src/lib.rs similarity index 100% rename from crates/deltalake-sql/src/lib.rs rename to crates/sql/src/lib.rs diff --git a/crates/deltalake-sql/src/logical_plan.rs b/crates/sql/src/logical_plan.rs similarity index 100% rename from crates/deltalake-sql/src/logical_plan.rs rename to crates/sql/src/logical_plan.rs diff --git a/crates/deltalake-sql/src/parser.rs b/crates/sql/src/parser.rs similarity index 75% rename from crates/deltalake-sql/src/parser.rs rename to crates/sql/src/parser.rs index 3287c87215..10e7252730 100644 --- a/crates/deltalake-sql/src/parser.rs +++ b/crates/sql/src/parser.rs @@ -1,7 +1,7 @@ use std::collections::VecDeque; use std::fmt; -use datafusion_sql::parser::{DFParser, DescribeTableStmt, Statement as DFStatement}; +use datafusion_sql::parser::{DFParser, Statement as DFStatement}; use datafusion_sql::sqlparser::ast::{ObjectName, Value}; use datafusion_sql::sqlparser::dialect::{keywords::Keyword, Dialect, GenericDialect}; use datafusion_sql::sqlparser::parser::{Parser, ParserError}; @@ -138,10 +138,6 @@ impl<'a> DeltaParser<'a> { match self.parser.peek_token().token { Token::Word(w) => { match w.keyword { - Keyword::DESCRIBE => { - self.parser.next_token(); - self.parse_describe() - } Keyword::VACUUM => { self.parser.next_token(); self.parse_vacuum() @@ -167,50 +163,6 @@ impl<'a> DeltaParser<'a> { } } - /// Parse a SQL `DESCRIBE` statement - pub fn parse_describe(&mut self) -> Result { - match self.parser.peek_token().token { - Token::Word(w) => match w.keyword { - Keyword::DETAIL => { - self.parser.next_token(); - let table = self.parser.parse_object_name()?; - Ok(Statement::Describe(DescribeStatement { - table, - operation: DescribeOperation::Detail, - })) - } - Keyword::HISTORY => { - self.parser.next_token(); - let table = self.parser.parse_object_name()?; - Ok(Statement::Describe(DescribeStatement { - table, - operation: DescribeOperation::History, - })) - } - Keyword::FILES => { - self.parser.next_token(); - let table = self.parser.parse_object_name()?; - Ok(Statement::Describe(DescribeStatement { - table, - operation: DescribeOperation::Files, - })) - } - _ => { - let table = self.parser.parse_object_name()?; - Ok(Statement::Datafusion(DFStatement::DescribeTableStmt( - DescribeTableStmt { table_name: table }, - ))) - } - }, - _ => { - let table_name = self.parser.parse_object_name()?; - Ok(Statement::Datafusion(DFStatement::DescribeTableStmt( - DescribeTableStmt { table_name }, - ))) - } - } - } - pub fn parse_vacuum(&mut self) -> Result { let table_name = self.parser.parse_object_name()?; match self.parser.peek_token().token { @@ -287,44 +239,6 @@ mod tests { Ok(()) } - #[test] - fn test_parse_describe() { - let stmt = Statement::Describe(DescribeStatement { - table: ObjectName(vec![Ident { - value: "data_table".to_string(), - quote_style: None, - }]), - operation: DescribeOperation::History, - }); - assert!(expect_parse_ok("DESCRIBE HISTORY data_table", stmt).is_ok()); - - let stmt = Statement::Describe(DescribeStatement { - table: ObjectName(vec![Ident { - value: "data_table".to_string(), - quote_style: None, - }]), - operation: DescribeOperation::Detail, - }); - assert!(expect_parse_ok("DESCRIBE DETAIL data_table", stmt).is_ok()); - - let stmt = Statement::Describe(DescribeStatement { - table: ObjectName(vec![Ident { - value: "data_table".to_string(), - quote_style: None, - }]), - operation: DescribeOperation::Files, - }); - assert!(expect_parse_ok("DESCRIBE FILES data_table", stmt).is_ok()); - - let stmt = Statement::Datafusion(DFStatement::DescribeTableStmt(DescribeTableStmt { - table_name: ObjectName(vec![Ident { - value: "data_table".to_string(), - quote_style: None, - }]), - })); - assert!(expect_parse_ok("DESCRIBE data_table", stmt).is_ok()) - } - #[test] fn test_parse_vacuum() { let stmt = Statement::Vacuum(VacuumStatement { diff --git a/crates/deltalake-sql/src/planner.rs b/crates/sql/src/planner.rs similarity index 99% rename from crates/deltalake-sql/src/planner.rs rename to crates/sql/src/planner.rs index 3eb4742308..099f97087d 100644 --- a/crates/deltalake-sql/src/planner.rs +++ b/crates/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_expr::logical_plan::builder::LogicalTableSource; use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource}; use datafusion_sql::TableReference; diff --git a/crates/deltalake-core/.gitignore b/crates/test/.gitignore similarity index 100% rename from crates/deltalake-core/.gitignore rename to crates/test/.gitignore diff --git a/crates/test/Cargo.toml b/crates/test/Cargo.toml new file mode 100644 index 0000000000..bca9094150 --- /dev/null +++ b/crates/test/Cargo.toml @@ -0,0 +1,23 @@ +[package] +name = "deltalake-test" +version = "0.1.0" +edition = "2021" +publish = false + +[dependencies] +bytes = { workspace = true } +chrono = { workspace = true, default-features = false, features = ["clock"] } +deltalake-core = { version = "0.17.0", path = "../core" } +dotenvy = "0" +fs_extra = "1.3.0" +futures = { version = "0.3" } +object_store = { workspace = true } +rand = "0.8" +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true } +tempfile = "3" +tokio = { version = "1", features = ["macros", "rt-multi-thread"] } + +[features] +default = [] +datafusion = ["deltalake-core/datafusion"] diff --git a/crates/deltalake-core/tests/common/clock.rs b/crates/test/src/clock.rs similarity index 100% rename from crates/deltalake-core/tests/common/clock.rs rename to crates/test/src/clock.rs diff --git a/crates/deltalake-core/tests/integration_concurrent_writes.rs b/crates/test/src/concurrent.rs similarity index 71% rename from crates/deltalake-core/tests/integration_concurrent_writes.rs rename to crates/test/src/concurrent.rs index 67fbbe38c5..646fb4e14f 100644 --- a/crates/deltalake-core/tests/integration_concurrent_writes.rs +++ b/crates/test/src/concurrent.rs @@ -1,48 +1,18 @@ -#![cfg(feature = "integration_test")] - -use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; -use deltalake_core::operations::transaction::commit; -use deltalake_core::operations::DeltaOps; -use deltalake_core::protocol::{DeltaOperation, SaveMode}; -use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; -use deltalake_core::{DeltaTable, DeltaTableBuilder}; use std::collections::HashMap; use std::future::Future; use std::iter::FromIterator; use std::time::Duration; -#[tokio::test] -async fn test_concurrent_writes_local() -> TestResult { - test_concurrent_writes(StorageIntegration::Local).await?; - Ok(()) -} - -#[cfg(feature = "s3")] -#[tokio::test] -async fn concurrent_writes_s3() -> TestResult { - test_concurrent_writes(StorageIntegration::Amazon).await?; - Ok(()) -} - -#[cfg(feature = "azure")] -#[tokio::test] -async fn test_concurrent_writes_azure() -> TestResult { - test_concurrent_writes(StorageIntegration::Microsoft).await?; - Ok(()) -} +use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; +use deltalake_core::operations::transaction::CommitBuilder; +use deltalake_core::operations::DeltaOps; +use deltalake_core::protocol::{DeltaOperation, SaveMode}; +use deltalake_core::{DeltaTable, DeltaTableBuilder}; -// tracked via https://github.com/datafusion-contrib/datafusion-objectstore-hdfs/issues/13 -#[ignore] -#[cfg(feature = "hdfs")] -#[tokio::test] -async fn test_concurrent_writes_hdfs() -> TestResult { - test_concurrent_writes(StorageIntegration::Hdfs).await?; - Ok(()) -} +use crate::utils::*; -async fn test_concurrent_writes(integration: StorageIntegration) -> TestResult { - let context = IntegrationContext::new(integration)?; - let (_table, table_uri) = prepare_table(&context).await?; +pub async fn test_concurrent_writes(context: &IntegrationContext) -> TestResult { + let (_table, table_uri) = prepare_table(context).await?; run_test(|name| Worker::new(&table_uri, name)).await; Ok(()) } @@ -68,9 +38,9 @@ async fn prepare_table( .await?; assert_eq!(0, table.version()); - assert_eq!(1, table.protocol().min_reader_version); - assert_eq!(2, table.protocol().min_writer_version); - assert_eq!(0, table.get_files().len()); + assert_eq!(1, table.protocol()?.min_reader_version); + assert_eq!(2, table.protocol()?.min_writer_version); + // assert_eq!(0, table.get_files_iter().count()); Ok((table, table_uri)) } @@ -157,7 +127,6 @@ impl Worker { path: format!("{}.parquet", name), size: 396, partition_values: HashMap::new(), - partition_values_parsed: None, modification_time: 1564524294000, data_change: true, stats: None, @@ -168,15 +137,15 @@ impl Worker { default_row_commit_version: None, clustering_provider: None, })]; - let version = commit( - self.table.log_store().as_ref(), - &actions, - operation, - &self.table.state, - None, - ) - .await - .unwrap(); + let snapshot = self.table.snapshot().unwrap().snapshot(); + + let version = CommitBuilder::default() + .with_actions(actions) + .build(Some(snapshot), self.table.log_store(), operation) + .unwrap() + .await + .unwrap() + .version(); self.table.update().await.unwrap(); version } diff --git a/crates/deltalake-core/tests/common/datafusion.rs b/crates/test/src/datafusion.rs similarity index 68% rename from crates/deltalake-core/tests/common/datafusion.rs rename to crates/test/src/datafusion.rs index 67c9ac442b..8207233ef9 100644 --- a/crates/deltalake-core/tests/common/datafusion.rs +++ b/crates/test/src/datafusion.rs @@ -1,6 +1,6 @@ -use datafusion::execution::context::{SessionContext, SessionState}; -use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; -use datafusion::prelude::SessionConfig; +use deltalake_core::datafusion::execution::context::{SessionContext, SessionState}; +use deltalake_core::datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use deltalake_core::datafusion::prelude::SessionConfig; use deltalake_core::delta_datafusion::DeltaTableFactory; use std::sync::Arc; diff --git a/crates/deltalake-core/tests/common/mod.rs b/crates/test/src/lib.rs similarity index 86% rename from crates/deltalake-core/tests/common/mod.rs rename to crates/test/src/lib.rs index d2742b2718..f6ac7727f5 100644 --- a/crates/deltalake-core/tests/common/mod.rs +++ b/crates/test/src/lib.rs @@ -4,25 +4,26 @@ use bytes::Bytes; use deltalake_core::kernel::{Action, Add, Remove, StructType}; use deltalake_core::logstore::LogStore; use deltalake_core::operations::create::CreateBuilder; -use deltalake_core::operations::transaction::commit; +use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use deltalake_core::DeltaTable; use deltalake_core::DeltaTableBuilder; -use object_store::{path::Path, ObjectStore}; +use deltalake_core::{ObjectStore, Path}; use std::any::Any; use std::collections::HashMap; use std::sync::Arc; -use tempdir::TempDir; +use tempfile::TempDir; -#[cfg(feature = "azure")] -pub mod adls; pub mod clock; +pub mod concurrent; #[cfg(feature = "datafusion")] pub mod datafusion; -#[cfg(feature = "hdfs")] -pub mod hdfs; -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -pub mod s3; +pub mod read; +pub mod utils; + +pub use concurrent::test_concurrent_writes; +pub use read::*; +pub use utils::{IntegrationContext, TestResult}; #[derive(Default)] pub struct TestContext { @@ -47,8 +48,6 @@ impl TestContext { Ok("LOCALFS") | Err(std::env::VarError::NotPresent) => setup_local_context().await, #[cfg(feature = "azure")] Ok("AZURE_GEN2") => adls::setup_azure_gen2_context().await, - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - Ok("S3_LOCAL_STACK") => s3::setup_s3_context().await, #[cfg(feature = "hdfs")] Ok("HDFS") => hdfs::setup_hdfs_context(), _ => panic!("Invalid backend for delta-rs tests"), @@ -95,7 +94,7 @@ impl TestContext { } pub async fn setup_local_context() -> TestContext { - let tmp_dir = tempdir::TempDir::new("delta-rs_tests").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let mut config = HashMap::new(); config.insert( "URI".to_owned(), @@ -136,7 +135,6 @@ pub async fn add_file( data_change: true, stats: None, stats_parsed: None, - partition_values_parsed: None, tags: None, default_row_commit_version: None, base_row_id: None, @@ -149,15 +147,14 @@ pub async fn add_file( predicate: None, }; let actions = vec![Action::Add(add)]; - commit( - table.log_store().as_ref(), - &actions, - operation, - &table.state, - None, - ) - .await - .unwrap(); + let snapshot = table.snapshot().unwrap().snapshot(); + + CommitBuilder::default() + .with_actions(actions) + .build(Some(snapshot), table.log_store(), operation) + .unwrap() + .await + .unwrap(); table.update().await.unwrap(); } } @@ -187,14 +184,13 @@ pub async fn remove_file( }; let operation = DeltaOperation::Delete { predicate: None }; let actions = vec![Action::Remove(remove)]; - commit( - table.log_store().as_ref(), - &actions, - operation, - &table.state, - None, - ) - .await - .unwrap(); + let snapshot = table.snapshot().unwrap().snapshot(); + + CommitBuilder::default() + .with_actions(actions) + .build(Some(snapshot), table.log_store(), operation) + .unwrap() + .await + .unwrap(); table.update().await.unwrap(); } diff --git a/crates/test/src/read.rs b/crates/test/src/read.rs new file mode 100644 index 0000000000..b3942a04b4 --- /dev/null +++ b/crates/test/src/read.rs @@ -0,0 +1,172 @@ +use deltalake_core::DeltaTableBuilder; +use object_store::path::Path; + +use crate::utils::{IntegrationContext, TestResult, TestTables}; + +pub async fn test_read_tables(context: &IntegrationContext) -> TestResult { + context.load_table(TestTables::Simple).await?; + context.load_table(TestTables::Golden).await?; + context + .load_table(TestTables::Delta0_8_0SpecialPartitioned) + .await?; + + read_simple_table(context).await?; + read_simple_table_with_version(context).await?; + read_golden(context).await?; + + Ok(()) +} + +pub async fn read_table_paths( + context: &IntegrationContext, + table_root: &str, + upload_path: &str, +) -> TestResult { + context + .load_table_with_name(TestTables::Delta0_8_0SpecialPartitioned, upload_path) + .await?; + + verify_store(context, table_root).await?; + + read_encoded_table(context, table_root).await?; + + Ok(()) +} + +async fn read_simple_table(integration: &IntegrationContext) -> TestResult { + let table_uri = integration.uri_for_table(TestTables::Simple); + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .load() + .await?; + + assert_eq!(table.version(), 4); + assert_eq!(table.protocol()?.min_writer_version, 2); + assert_eq!(table.protocol()?.min_reader_version, 1); + assert_eq!( + table.get_files_iter()?.collect::>(), + vec![ + Path::from("part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"), + Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), + Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), + Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), + Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), + ] + ); + let tombstones = table + .snapshot()? + .all_tombstones(table.object_store().clone()) + .await? + .collect::>(); + assert_eq!(tombstones.len(), 31); + assert!(tombstones.contains(&deltalake_core::kernel::Remove { + path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), + deletion_timestamp: Some(1587968596250), + data_change: true, + extended_file_metadata: None, + deletion_vector: None, + base_row_id: None, + default_row_commit_version: None, + size: None, + partition_values: Some(Default::default()), + tags: Some(Default::default()), + })); + + Ok(()) +} + +async fn read_simple_table_with_version(integration: &IntegrationContext) -> TestResult { + let table_uri = integration.uri_for_table(TestTables::Simple); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .with_version(3) + .load() + .await?; + + assert_eq!(table.version(), 3); + assert_eq!(table.protocol()?.min_writer_version, 2); + assert_eq!(table.protocol()?.min_reader_version, 1); + assert_eq!( + table.get_files_iter()?.collect::>(), + vec![ + Path::from("part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet"), + Path::from("part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet"), + Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), + Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), + Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), + Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), + ] + ); + let tombstones = table + .snapshot()? + .all_tombstones(table.object_store().clone()) + .await? + .collect::>(); + assert_eq!(tombstones.len(), 29); + assert!(tombstones.contains(&deltalake_core::kernel::Remove { + path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), + deletion_timestamp: Some(1587968596250), + data_change: true, + tags: Some(Default::default()), + partition_values: Some(Default::default()), + base_row_id: None, + default_row_commit_version: None, + size: None, + deletion_vector: None, + extended_file_metadata: None, + })); + + Ok(()) +} + +pub async fn read_golden(integration: &IntegrationContext) -> TestResult { + let table_uri = integration.uri_for_table(TestTables::Golden); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .load() + .await + .unwrap(); + + assert_eq!(table.version(), 0); + assert_eq!(table.protocol()?.min_writer_version, 2); + assert_eq!(table.protocol()?.min_reader_version, 1); + + Ok(()) +} + +async fn verify_store(integration: &IntegrationContext, root_path: &str) -> TestResult { + let table_uri = format!("{}/{}", integration.root_uri(), root_path); + + let storage = DeltaTableBuilder::from_uri(table_uri.clone()) + .with_allow_http(true) + .build_storage()? + .object_store(); + + let files = storage.list_with_delimiter(None).await?; + assert_eq!( + vec![ + Path::parse("_delta_log").unwrap(), + Path::parse("x=A%2FA").unwrap(), + Path::parse("x=B%20B").unwrap(), + ], + files.common_prefixes + ); + + Ok(()) +} + +async fn read_encoded_table(integration: &IntegrationContext, root_path: &str) -> TestResult { + let table_uri = format!("{}/{}", integration.root_uri(), root_path); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_allow_http(true) + .load() + .await?; + + assert_eq!(table.version(), 0); + assert_eq!(table.get_files_iter()?.count(), 2); + + Ok(()) +} diff --git a/crates/test/src/utils.rs b/crates/test/src/utils.rs new file mode 100644 index 0000000000..b5102e2808 --- /dev/null +++ b/crates/test/src/utils.rs @@ -0,0 +1,298 @@ +#![allow(dead_code, missing_docs)] +use deltalake_core::storage::ObjectStoreRef; +use deltalake_core::{DeltaResult, DeltaTableBuilder}; +use fs_extra::dir::{copy, CopyOptions}; +use std::collections::HashMap; +use std::env; +use std::process::ExitStatus; +use tempfile::{tempdir, TempDir}; + +pub type TestResult = Result>; + +pub trait StorageIntegration { + fn create_bucket(&self) -> std::io::Result; + fn prepare_env(&self); + fn bucket_name(&self) -> String; + fn root_uri(&self) -> String; + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result; + + fn object_store(&self) -> DeltaResult { + Ok(DeltaTableBuilder::from_uri(self.root_uri()) + .with_allow_http(true) + .build_storage()? + .object_store()) + } +} + +pub struct LocalStorageIntegration { + tmp_dir: TempDir, +} + +impl Default for LocalStorageIntegration { + fn default() -> Self { + Self { + tmp_dir: tempdir().expect("Failed to make temp dir"), + } + } +} + +impl StorageIntegration for LocalStorageIntegration { + fn create_bucket(&self) -> std::io::Result { + Ok(ExitStatus::default()) + } + + fn prepare_env(&self) {} + fn bucket_name(&self) -> String { + self.tmp_dir.as_ref().to_str().unwrap().to_owned() + } + fn root_uri(&self) -> String { + format!("file://{}", self.bucket_name()) + } + fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { + let mut options = CopyOptions::new(); + options.content_only = true; + let dest_path = self.tmp_dir.path().join(destination); + std::fs::create_dir_all(&dest_path)?; + copy(source, &dest_path, &options).expect("Failed to copy"); + Ok(ExitStatus::default()) + } +} + +/// The IntegrationContext provides temporary resources to test against cloud storage services. +pub struct IntegrationContext { + pub integration: Box, + bucket: String, + store: ObjectStoreRef, + tmp_dir: TempDir, + /// environment variables valid before `prepare_env()` modified them + env_vars: HashMap, +} + +impl IntegrationContext { + pub fn new( + integration: Box, + ) -> Result> { + // environment variables are loaded from .env files if found. Otherwise + // default values based on the default setting of the respective emulators are set. + #[cfg(test)] + dotenvy::dotenv().ok(); + + // save existing environment variables + let env_vars = std::env::vars().collect(); + + integration.prepare_env(); + + let tmp_dir = tempdir()?; + // create a fresh bucket in every context. THis is done via CLI... + integration.create_bucket()?; + let store = integration.object_store()?; + let bucket = integration.bucket_name(); + + Ok(Self { + integration, + bucket, + store, + tmp_dir, + env_vars, + }) + } + + /// Get a a reference to the root object store + pub fn object_store(&self) -> ObjectStoreRef { + self.store.clone() + } + + /// Get the URI for initializing a store at the root + pub fn root_uri(&self) -> String { + self.integration.root_uri() + } + + pub fn table_builder(&self, table: TestTables) -> DeltaTableBuilder { + let name = table.as_name(); + let table_uri = format!("{}/{}", self.root_uri(), &name); + DeltaTableBuilder::from_uri(table_uri).with_allow_http(true) + } + + pub fn uri_for_table(&self, table: TestTables) -> String { + format!("{}/{}", self.root_uri(), table.as_name()) + } + + pub async fn load_table(&self, table: TestTables) -> TestResult { + let name = table.as_name(); + self.load_table_with_name(table, name).await + } + + pub async fn load_table_with_name( + &self, + table: TestTables, + name: impl AsRef, + ) -> TestResult { + self.integration + .copy_directory(&table.as_path(), name.as_ref())?; + Ok(()) + } + + fn restore_env(&self) { + let env_vars: HashMap<_, _> = std::env::vars().collect(); + for (key, _) in env_vars { + if !self.env_vars.contains_key(&key) { + std::env::remove_var(key) + } + } + for (key, value) in self.env_vars.iter() { + std::env::set_var(key, value); + } + } +} + +/// Reference tables from the test data folder +pub enum TestTables { + Simple, + SimpleWithCheckpoint, + SimpleCommit, + Golden, + Delta0_8_0Partitioned, + Delta0_8_0SpecialPartitioned, + Checkpoints, + WithDvSmall, + Custom(String), +} + +impl TestTables { + fn as_path(&self) -> String { + // env "CARGO_MANIFEST_DIR" is "the directory containing the manifest of your package", + // set by `cargo run` or `cargo test`, see: + // https://doc.rust-lang.org/cargo/reference/environment-variables.html + let dir = env!("CARGO_MANIFEST_DIR"); + let data_path = std::path::Path::new(dir).join("tests/data"); + match self { + Self::Simple => data_path.join("simple_table").to_str().unwrap().to_owned(), + Self::SimpleWithCheckpoint => data_path + .join("simple_table_with_checkpoint") + .to_str() + .unwrap() + .to_owned(), + Self::SimpleCommit => data_path.join("simple_commit").to_str().unwrap().to_owned(), + Self::Golden => data_path + .join("golden/data-reader-array-primitives") + .to_str() + .unwrap() + .to_owned(), + Self::Delta0_8_0Partitioned => data_path + .join("delta-0.8.0-partitioned") + .to_str() + .unwrap() + .to_owned(), + Self::Delta0_8_0SpecialPartitioned => data_path + .join("delta-0.8.0-special-partition") + .to_str() + .unwrap() + .to_owned(), + Self::Checkpoints => data_path.join("checkpoints").to_str().unwrap().to_owned(), + Self::WithDvSmall => data_path + .join("table-with-dv-small") + .to_str() + .unwrap() + .to_owned(), + // the data path for upload does not apply to custom tables. + Self::Custom(_) => todo!(), + } + } + + pub fn as_name(&self) -> String { + match self { + Self::Simple => "simple".into(), + Self::SimpleWithCheckpoint => "simple_table_with_checkpoint".into(), + Self::SimpleCommit => "simple_commit".into(), + Self::Golden => "golden".into(), + Self::Delta0_8_0Partitioned => "delta-0.8.0-partitioned".into(), + Self::Delta0_8_0SpecialPartitioned => "delta-0.8.0-special-partition".into(), + Self::Checkpoints => "checkpoints".into(), + Self::WithDvSmall => "table-with-dv-small".into(), + Self::Custom(name) => name.to_owned(), + } + } +} + +/// Set environment variable if it is not set +pub fn set_env_if_not_set(key: impl AsRef, value: impl AsRef) { + if std::env::var(key.as_ref()).is_err() { + std::env::set_var(key.as_ref(), value.as_ref()) + }; +} + +/// small wrapper around hdfs cli +pub mod hdfs_cli { + use std::env; + use std::path::PathBuf; + use std::process::{Command, ExitStatus}; + + fn hdfs_cli_path() -> PathBuf { + let hadoop_home = + env::var("HADOOP_HOME").expect("HADOOP_HOME environment variable not set"); + PathBuf::from(hadoop_home).join("bin").join("hdfs") + } + + pub fn create_dir(dir_name: impl AsRef) -> std::io::Result { + let path = hdfs_cli_path(); + let mut child = Command::new(path) + .args([ + "dfs", + "-mkdir", + "-p", + format!("/{}", dir_name.as_ref()).as_str(), + ]) + .spawn() + .expect("hdfs command is installed"); + child.wait() + } + + pub fn delete_dir(dir_name: impl AsRef) -> std::io::Result { + let path = hdfs_cli_path(); + let mut child = Command::new(path) + .args([ + "dfs", + "-rm", + "-r", + "-f", + format!("/{}", dir_name.as_ref()).as_str(), + ]) + .spawn() + .expect("hdfs command is installed"); + child.wait() + } +} + +#[macro_export] +macro_rules! assert_batches_sorted_eq { + ($EXPECTED_LINES: expr, $CHUNKS: expr) => { + let mut expected_lines: Vec = $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); + + // sort except for header + footer + let num_lines = expected_lines.len(); + if num_lines > 3 { + expected_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + let formatted = arrow::util::pretty::pretty_format_batches($CHUNKS) + .unwrap() + .to_string(); + // fix for windows: \r\n --> + + let mut actual_lines: Vec<&str> = formatted.trim().lines().collect(); + + // sort except for header + footer + let num_lines = actual_lines.len(); + if num_lines > 3 { + actual_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + assert_eq!( + expected_lines, actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +pub use assert_batches_sorted_eq; diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 b/crates/test/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 rename to crates/test/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 b/crates/test/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 rename to crates/test/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 b/crates/test/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 rename to crates/test/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc b/crates/test/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc rename to crates/test/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json b/crates/test/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json rename to crates/test/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet b/crates/test/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet rename to crates/test/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json b/crates/test/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json rename to crates/test/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet b/crates/test/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet rename to crates/test/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json b/crates/test/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json rename to crates/test/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint b/crates/test/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint similarity index 100% rename from crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint rename to crates/test/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/.gitignore b/crates/test/tests/data/checkpoints/_delta_log/.gitignore similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/.gitignore rename to crates/test/tests/data/checkpoints/_delta_log/.gitignore diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000000.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000000.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000001.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000001.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000002.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000002.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000003.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000003.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000004.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000004.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000004.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000004.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000005.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000005.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000005.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000005.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000006.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000006.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000006.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000006.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000007.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000007.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000007.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000007.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000008.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000008.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000008.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000008.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000009.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000009.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000009.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000009.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000010.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000010.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000010.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000010.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000011.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000011.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000011.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000011.json diff --git a/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000012.json b/crates/test/tests/data/checkpoints/_delta_log/00000000000000000012.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000012.json rename to crates/test/tests/data/checkpoints/_delta_log/00000000000000000012.json diff --git a/crates/deltalake-core/tests/data/checkpoints_tombstones/.gitignore b/crates/test/tests/data/checkpoints_tombstones/.gitignore similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_tombstones/.gitignore rename to crates/test/tests/data/checkpoints_tombstones/.gitignore diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json diff --git a/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json b/crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json similarity index 100% rename from crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json rename to crates/test/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json diff --git a/crates/deltalake-core/tests/data/concurrent_workers/_delta_log/.gitignore b/crates/test/tests/data/concurrent_workers/_delta_log/.gitignore similarity index 100% rename from crates/deltalake-core/tests/data/concurrent_workers/_delta_log/.gitignore rename to crates/test/tests/data/concurrent_workers/_delta_log/.gitignore diff --git a/crates/deltalake-core/tests/data/concurrent_workers/_delta_log/00000000000000000000.json b/crates/test/tests/data/concurrent_workers/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/concurrent_workers/_delta_log/00000000000000000000.json rename to crates/test/tests/data/concurrent_workers/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc b/crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc rename to crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc b/crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc rename to crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc b/crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc rename to crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc b/crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc rename to crates/test/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc b/crates/test/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc rename to crates/test/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc b/crates/test/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc rename to crates/test/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json b/crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json rename to crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json b/crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json rename to crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet b/crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet rename to crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json b/crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json rename to crates/test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/_last_checkpoint b/crates/test/tests/data/delta-0.2.0/_delta_log/_last_checkpoint similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/_last_checkpoint rename to crates/test/tests/data/delta-0.2.0/_delta_log/_last_checkpoint diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet b/crates/test/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet rename to crates/test/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json b/crates/test/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json rename to crates/test/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet b/crates/test/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet rename to crates/test/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet b/crates/test/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet rename to crates/test/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-null-partition/k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-null-partition/k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-null-partition/k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-null-partition/k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/.part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/.part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/.part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/.part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-numeric-partition/x=10/y=10.0/part-00015-24eb4845-2d25-4448-b3bb-5ed7f12635ab.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/.part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/.part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/.part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/.part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-numeric-partition/x=9/y=9.9/part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/.part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/.part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/.part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/.part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/.part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/.part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/.part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/.part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/.part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/.part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/.part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/.part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/.part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/.part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/.part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/.part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/.part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/.part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/.part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/.part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/.part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/.part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/.part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/.part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-partitioned/year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=A%2FA/.part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-special-partition/x=A%2FA/.part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=A%2FA/.part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-special-partition/x=A%2FA/.part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=B%20B/.part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet.crc b/crates/test/tests/data/delta-0.8.0-special-partition/x=B%20B/.part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=B%20B/.part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-0.8.0-special-partition/x=B%20B/.part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/_change_data/.gitkeep b/crates/test/tests/data/delta-0.8.0/_change_data/.gitkeep similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/_change_data/.gitkeep rename to crates/test/tests/data/delta-0.8.0/_change_data/.gitkeep diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/_delta_index/.gitkeep b/crates/test/tests/data/delta-0.8.0/_delta_index/.gitkeep similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/_delta_index/.gitkeep rename to crates/test/tests/data/delta-0.8.0/_delta_index/.gitkeep diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json b/crates/test/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json rename to crates/test/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet b/crates/test/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet rename to crates/test/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint b/crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet b/crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet rename to crates/test/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc b/crates/test/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc rename to crates/test/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/.part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet.crc b/crates/test/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/.part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/.part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/.part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet b/crates/test/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet rename to crates/test/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/.part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet.crc b/crates/test/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/.part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/.part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/.part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet b/crates/test/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet rename to crates/test/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/.part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet.crc b/crates/test/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/.part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/.part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet.crc rename to crates/test/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/.part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet b/crates/test/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet rename to crates/test/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000000.json b/crates/test/tests/data/delta-live-table/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000000.json rename to crates/test/tests/data/delta-live-table/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000001.json b/crates/test/tests/data/delta-live-table/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000001.json rename to crates/test/tests/data/delta-live-table/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc b/crates/test/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc rename to crates/test/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc b/crates/test/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc rename to crates/test/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json b/crates/test/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json rename to crates/test/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet b/crates/test/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet rename to crates/test/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet b/crates/test/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet rename to crates/test/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000000.json b/crates/test/tests/data/http_requests/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000000.json rename to crates/test/tests/data/http_requests/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000001.json b/crates/test/tests/data/http_requests/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000001.json rename to crates/test/tests/data/http_requests/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet b/crates/test/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet rename to crates/test/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet b/crates/test/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet rename to crates/test/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet diff --git a/crates/test/tests/data/issue-2152/_delta_log/00000000000000000000.json b/crates/test/tests/data/issue-2152/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..45256b6236 --- /dev/null +++ b/crates/test/tests/data/issue-2152/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1706770085847,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","description":null,"isManaged":"true","properties":"{}","statsOnLoad":false},"isolationLevel":"WriteSerializable","isBlindAppend":true,"operationMetrics":{},"tags":{"restoresDeletedRows":"false"},"engineInfo":"Databricks-Runtime/14.1.x-photon-scala2.12","txnId":"5ba2d1f4-09e0-4013-920a-92b057185128"}} +{"metaData":{"id":"7791991a-60e9-4a8f-bff0-ccffec779dc4","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"ID_D_DATE\",\"type\":\"long\",\"nullable\":true,\"metadata\":{\"delta.identity.start\":1,\"delta.identity.step\":1,\"delta.identity.allowExplicitInsert\":false}},{\"name\":\"TXT_DateKey\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1706770085202}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":6}} diff --git a/crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000000.json b/crates/test/tests/data/issue_1374/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000000.json rename to crates/test/tests/data/issue_1374/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet b/crates/test/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet rename to crates/test/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.json b/crates/test/tests/data/issue_1374/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.json rename to crates/test/tests/data/issue_1374/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/issue_1374/_delta_log/_last_checkpoint b/crates/test/tests/data/issue_1374/_delta_log/_last_checkpoint similarity index 100% rename from crates/deltalake-core/tests/data/issue_1374/_delta_log/_last_checkpoint rename to crates/test/tests/data/issue_1374/_delta_log/_last_checkpoint diff --git a/crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet b/crates/test/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet rename to crates/test/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet b/crates/test/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet rename to crates/test/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc b/crates/test/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc b/crates/test/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc b/crates/test/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc b/crates/test/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_commit/_delta_log/.gitignore b/crates/test/tests/data/simple_commit/_delta_log/.gitignore similarity index 100% rename from crates/deltalake-core/tests/data/simple_commit/_delta_log/.gitignore rename to crates/test/tests/data/simple_commit/_delta_log/.gitignore diff --git a/crates/deltalake-core/tests/data/simple_commit/_delta_log/00000000000000000000.json b/crates/test/tests/data/simple_commit/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_commit/_delta_log/00000000000000000000.json rename to crates/test/tests/data/simple_commit/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet b/crates/test/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet rename to crates/test/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet b/crates/test/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet rename to crates/test/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet b/crates/test/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet rename to crates/test/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet b/crates/test/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet rename to crates/test/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000000.json b/crates/test/tests/data/simple_table/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000000.json rename to crates/test/tests/data/simple_table/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000001.json b/crates/test/tests/data/simple_table/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000001.json rename to crates/test/tests/data/simple_table/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000002.json b/crates/test/tests/data/simple_table/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000002.json rename to crates/test/tests/data/simple_table/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000003.json b/crates/test/tests/data/simple_table/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000003.json rename to crates/test/tests/data/simple_table/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000004.json b/crates/test/tests/data/simple_table/_delta_log/00000000000000000004.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000004.json rename to crates/test/tests/data/simple_table/_delta_log/00000000000000000004.json diff --git a/crates/deltalake-core/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet b/crates/test/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet rename to crates/test/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc b/crates/test/tests/data/simple_table_features/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc rename to crates/test/tests/data/simple_table_features/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000000.json b/crates/test/tests/data/simple_table_features/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000000.json rename to crates/test/tests/data/simple_table_features/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000001.json b/crates/test/tests/data/simple_table_features/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000001.json rename to crates/test/tests/data/simple_table_features/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000002.json b/crates/test/tests/data/simple_table_features/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000002.json rename to crates/test/tests/data/simple_table_features/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000003.json b/crates/test/tests/data/simple_table_features/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000003.json rename to crates/test/tests/data/simple_table_features/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000004.json b/crates/test/tests/data/simple_table_features/_delta_log/00000000000000000004.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000004.json rename to crates/test/tests/data/simple_table_features/_delta_log/00000000000000000004.json diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet b/crates/test/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet rename to crates/test/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet b/crates/test/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc b/crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc rename to crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json b/crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json rename to crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc b/crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc rename to crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json b/crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json rename to crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc b/crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc rename to crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json b/crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json rename to crates/test/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet b/crates/test/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint b/crates/test/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint rename to crates/test/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet b/crates/test/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet rename to crates/test/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table-with-dv-small/_delta_log/00000000000000000000.json b/crates/test/tests/data/table-with-dv-small/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/table-with-dv-small/_delta_log/00000000000000000000.json rename to crates/test/tests/data/table-with-dv-small/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/table-with-dv-small/_delta_log/00000000000000000001.json b/crates/test/tests/data/table-with-dv-small/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/table-with-dv-small/_delta_log/00000000000000000001.json rename to crates/test/tests/data/table-with-dv-small/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin b/crates/test/tests/data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin similarity index 100% rename from crates/deltalake-core/tests/data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin rename to crates/test/tests/data/table-with-dv-small/deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin diff --git a/crates/deltalake-core/tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet b/crates/test/tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet rename to crates/test/tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table-without-dv-small/_delta_log/00000000000000000000.json b/crates/test/tests/data/table-without-dv-small/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/table-without-dv-small/_delta_log/00000000000000000000.json rename to crates/test/tests/data/table-without-dv-small/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/table-without-dv-small/part-00000-517f5d32-9c95-48e8-82b4-0229cc194867-c000.snappy.parquet b/crates/test/tests/data/table-without-dv-small/part-00000-517f5d32-9c95-48e8-82b4-0229cc194867-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table-without-dv-small/part-00000-517f5d32-9c95-48e8-82b4-0229cc194867-c000.snappy.parquet rename to crates/test/tests/data/table-without-dv-small/part-00000-517f5d32-9c95-48e8-82b4-0229cc194867-c000.snappy.parquet diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000000.json b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..ae3330f9ab --- /dev/null +++ b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"98c9faeb-7940-43eb-9898-50b2a99c0a7e","name":null,"description":null,"format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"price\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"sold\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"deleted\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1709986334419,"configuration":{}}} +{"add":{"path":"part-00001-9c90a84d-6999-463c-bd2d-f68333e6d03d-c000.snappy.parquet","partitionValues":{},"size":1432,"modificationTime":1709986334424,"dataChange":true,"stats":"{\"numRecords\":5,\"minValues\":{\"deleted\":false,\"id\":\"1\",\"sold\":0,\"price\":0},\"maxValues\":{\"price\":4,\"id\":\"5\",\"deleted\":false,\"sold\":4},\"nullCount\":{\"deleted\":0,\"id\":0,\"price\":0,\"sold\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}} +{"commitInfo":{"timestamp":1709986334424,"operation":"WRITE","operationParameters":{"mode":"Append"},"clientVersion":"delta-rs.0.17.1"}} \ No newline at end of file diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000001.checkpoint.parquet b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000001.checkpoint.parquet new file mode 100644 index 0000000000..86af3c9c47 Binary files /dev/null and b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000001.checkpoint.parquet differ diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000001.json b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..b4d6ce3d67 --- /dev/null +++ b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"add":{"path":"part-00001-fed6d112-d244-4c54-810d-25ba3f0a4016-c000.snappy.parquet","partitionValues":{},"size":1432,"modificationTime":1709986334474,"dataChange":true,"stats":"{\"numRecords\":5,\"minValues\":{\"deleted\":false,\"price\":0,\"id\":\"1\",\"sold\":0},\"maxValues\":{\"price\":4,\"id\":\"5\",\"sold\":4,\"deleted\":false},\"nullCount\":{\"id\":0,\"sold\":0,\"deleted\":0,\"price\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}} +{"commitInfo":{"timestamp":1709986334475,"operation":"WRITE","operationParameters":{"mode":"Append"},"clientVersion":"delta-rs.0.17.1"}} \ No newline at end of file diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000002.json b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..6df3799b4b --- /dev/null +++ b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"add":{"path":"part-00001-6791b37e-f318-4d2b-87a0-89be205c338b-c000.snappy.parquet","partitionValues":{},"size":1432,"modificationTime":1709986423857,"dataChange":true,"stats":"{\"numRecords\":5,\"minValues\":{\"deleted\":false,\"price\":0,\"id\":\"1\",\"sold\":0},\"maxValues\":{\"sold\":4,\"id\":\"5\",\"price\":4,\"deleted\":false},\"nullCount\":{\"id\":0,\"price\":0,\"sold\":0,\"deleted\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}} +{"commitInfo":{"timestamp":1709986423857,"operation":"WRITE","operationParameters":{"mode":"Append"},"clientVersion":"delta-rs.0.17.1"}} \ No newline at end of file diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000003.checkpoint.parquet b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000003.checkpoint.parquet new file mode 100644 index 0000000000..cf46a2973e Binary files /dev/null and b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000003.checkpoint.parquet differ diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000003.json b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..2ea49db4b5 --- /dev/null +++ b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"add":{"path":"part-00001-bea93a33-9112-41a5-aca6-c2d1f2c43873-c000.snappy.parquet","partitionValues":{},"size":1432,"modificationTime":1709986423962,"dataChange":true,"stats":"{\"numRecords\":5,\"minValues\":{\"id\":\"1\",\"sold\":0,\"deleted\":false,\"price\":0},\"maxValues\":{\"deleted\":false,\"id\":\"5\",\"price\":4,\"sold\":4},\"nullCount\":{\"id\":0,\"price\":0,\"sold\":0,\"deleted\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}} +{"commitInfo":{"timestamp":1709986423962,"operation":"WRITE","operationParameters":{"mode":"Append"},"clientVersion":"delta-rs.0.17.1"}} \ No newline at end of file diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/_last_checkpoint b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..6869291ae5 --- /dev/null +++ b/crates/test/tests/data/table_failed_last_checkpoint_update/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"size":6,"size_in_bytes":23521,"version":1} \ No newline at end of file diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-6791b37e-f318-4d2b-87a0-89be205c338b-c000.snappy.parquet b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-6791b37e-f318-4d2b-87a0-89be205c338b-c000.snappy.parquet new file mode 100644 index 0000000000..81742b7049 Binary files /dev/null and b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-6791b37e-f318-4d2b-87a0-89be205c338b-c000.snappy.parquet differ diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-9c90a84d-6999-463c-bd2d-f68333e6d03d-c000.snappy.parquet b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-9c90a84d-6999-463c-bd2d-f68333e6d03d-c000.snappy.parquet new file mode 100644 index 0000000000..0d502583fa Binary files /dev/null and b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-9c90a84d-6999-463c-bd2d-f68333e6d03d-c000.snappy.parquet differ diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-bea93a33-9112-41a5-aca6-c2d1f2c43873-c000.snappy.parquet b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-bea93a33-9112-41a5-aca6-c2d1f2c43873-c000.snappy.parquet new file mode 100644 index 0000000000..81742b7049 Binary files /dev/null and b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-bea93a33-9112-41a5-aca6-c2d1f2c43873-c000.snappy.parquet differ diff --git a/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-fed6d112-d244-4c54-810d-25ba3f0a4016-c000.snappy.parquet b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-fed6d112-d244-4c54-810d-25ba3f0a4016-c000.snappy.parquet new file mode 100644 index 0000000000..81742b7049 Binary files /dev/null and b/crates/test/tests/data/table_failed_last_checkpoint_update/part-00001-fed6d112-d244-4c54-810d-25ba3f0a4016-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet b/crates/test/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet rename to crates/test/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet b/crates/test/tests/data/table_with_column_mapping/BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_column_mapping/BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet rename to crates/test/tests/data/table_with_column_mapping/BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc b/crates/test/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc rename to crates/test/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json b/crates/test/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json rename to crates/test/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json b/crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint b/crates/test/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint rename to crates/test/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin b/crates/test/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin rename to crates/test/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin b/crates/test/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin rename to crates/test/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin diff --git a/crates/deltalake-core/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet b/crates/test/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet rename to crates/test/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc b/crates/test/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc rename to crates/test/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc diff --git a/crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json b/crates/test/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json rename to crates/test/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet b/crates/test/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet rename to crates/test/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet b/crates/test/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet rename to crates/test/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 b/crates/test/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 rename to crates/test/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 b/crates/test/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 rename to crates/test/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 b/crates/test/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 rename to crates/test/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc b/crates/test/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc rename to crates/test/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json b/crates/test/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json rename to crates/test/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet b/crates/test/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet similarity index 100% rename from crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet rename to crates/test/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet diff --git a/crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000000.json b/crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000000.json similarity index 100% rename from crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000000.json rename to crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000001.json b/crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000001.json similarity index 100% rename from crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000001.json rename to crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000001.json diff --git a/crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.checkpoint.parquet b/crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.checkpoint.parquet similarity index 100% rename from crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.checkpoint.parquet rename to crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.checkpoint.parquet diff --git a/crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.json b/crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.json similarity index 100% rename from crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.json rename to crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000002.json diff --git a/crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000003.json b/crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000003.json similarity index 100% rename from crates/deltalake-core/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000003.json rename to crates/test/tests/data/with_checkpoint_no_last_checkpoint/_delta_log/00000000000000000003.json diff --git a/delta-inspect/Cargo.toml b/delta-inspect/Cargo.toml index cc6efaa381..78d2477486 100644 --- a/delta-inspect/Cargo.toml +++ b/delta-inspect/Cargo.toml @@ -22,5 +22,5 @@ features = ["azure", "gcs"] [features] default = ["rustls"] -native-tls = ["deltalake/s3-native-tls", "deltalake/glue-native-tls"] +native-tls = ["deltalake/s3-native-tls", "deltalake/glue"] rustls = ["deltalake/s3", "deltalake/glue"] diff --git a/delta-inspect/src/main.rs b/delta-inspect/src/main.rs index 86a7b499bd..6dd63e065f 100644 --- a/delta-inspect/src/main.rs +++ b/delta-inspect/src/main.rs @@ -67,9 +67,9 @@ async fn main() -> anyhow::Result<()> { }; if files_matches.is_present("full_uri") { - table.get_file_uris().for_each(|f| println!("{f}")); + table.get_file_uris()?.for_each(|f| println!("{f}")); } else { - table.get_files_iter().for_each(|f| println!("{f}")); + table.get_files_iter()?.for_each(|f| println!("{f}")); }; } Some(("info", info_matches)) => { diff --git a/dev/release/update_change_log.sh b/dev/release/update_change_log.sh index 4928b6c061..a9e7d67481 100755 --- a/dev/release/update_change_log.sh +++ b/dev/release/update_change_log.sh @@ -16,8 +16,8 @@ set -e LANGUAGE="rust" -SINCE_VERSION=${SINCE_VERSION:-"0.6.0"} -FUTURE_RELEASE=${FUTURE_RELEASE:-"0.7.0"} +SINCE_VERSION=${SINCE_VERSION:-"0.16.0"} +FUTURE_RELEASE=${FUTURE_RELEASE:-"0.18.0"} # only consider tags of the correct language if [ "$LANGUAGE" == "rust" ]; then diff --git a/docs/_build/links.yml b/docs/_build/links.yml index 605cda2d55..5a603059bc 100644 --- a/docs/_build/links.yml +++ b/docs/_build/links.yml @@ -1,4 +1,6 @@ python: DeltaTable: PYTHON_API_URL/delta_table + replaceWhere: https://delta-io.github.io/delta-rs/api/delta_writer/ rust: - DeltaTable: https://docs.rs/deltalake/latest/deltalake/table/struct.DeltaTable.html \ No newline at end of file + DeltaTable: https://docs.rs/deltalake/latest/deltalake/table/struct.DeltaTable.html + replaceWhere: https://docs.rs/deltalake/latest/deltalake/operations/write/struct.WriteBuilder.html#method.with_replace_where \ No newline at end of file diff --git a/docs/api/delta_table/delta_table_alterer.md b/docs/api/delta_table/delta_table_alterer.md new file mode 100644 index 0000000000..d859f605e1 --- /dev/null +++ b/docs/api/delta_table/delta_table_alterer.md @@ -0,0 +1,11 @@ +--- +search: + boost: 10 +--- + + +# TableAlterer + +::: deltalake.table.TableAlterer + options: + show_root_heading: true \ No newline at end of file diff --git a/docs/api/delta_writer.md b/docs/api/delta_writer.md index 432a32b768..9b395d3604 100644 --- a/docs/api/delta_writer.md +++ b/docs/api/delta_writer.md @@ -8,6 +8,8 @@ search: ::: deltalake.write_deltalake +::: deltalake.WriterProperties + ## Convert to Delta Tables ::: deltalake.convert_to_deltalake diff --git a/docs/delta-interop.png b/docs/delta-interop.png new file mode 100644 index 0000000000..7eb4abfd2c Binary files /dev/null and b/docs/delta-interop.png differ diff --git a/docs/delta-lake-best-practices.md b/docs/delta-lake-best-practices.md new file mode 100644 index 0000000000..c83c7db7a3 --- /dev/null +++ b/docs/delta-lake-best-practices.md @@ -0,0 +1,121 @@ +# Delta Lake Best Practices + +This page outlines Delta Lake best practices. + +You should consider several factors to optimize the performance of your Delta tables and minimize costs. + +The Delta Lake best practices depend on your data ingestion into the Delta table and query patterns. You must understand your data and how users run queries to best leverage Delta Lake. + +## Compacting small files + +Delta tables work best when the files are “right-sized”. Files that are too small create I/O overhead. Files that are too large limit the impact of file skipping (a critical query optimization). + +Delta tables can accumulate a lot of small files, especially if you’re frequently writing small amounts of data. If your table has many small files, you should run a small compaction operation to consolidate all the tiny files into “right-sized” files. + +It’s generally best for files in a Delta table to be between 100MB and 1GB, but that can vary based on the overall size of the table and the query patterns. + +Delta Lake makes it easy to [compact the small files](https://delta-io.github.io/delta-rs/usage/optimize/small-file-compaction-with-optimize/). + +## Optimizing table layout + +You can colocate similar data in the same files to make file skipping more effective. Delta Lake supports [Z Ordering](https://delta-io.github.io/delta-rs/usage/optimize/delta-lake-z-order/), which can colocate similar data in the same files. + +Z Ordering can yield impressive performance gains for low-cardinality columns but also works well for high-cardinality columns. This is an advantage compared to Hive-style partitioning, which is only suitable for low-cardinality columns. + +You must analyze the most common query patterns and Z Order your dataset based on the columns allowing the most file skipping. The ability to colocate data in the same files decreases when you add more Z Order columns. + +Let’s look at Hive-style partitioning, another way to colocate data in the same files. You can also use Hive-style partitioning in conjunction with Z Ordering. + +## Partitioning datasets + +You can partition your Delta tables, which separates the data by one or more partition keys into separate folders. Partitioning can be an excellent performance optimization (when you filter on the partition key) and is a good way to sidestep concurrency conflict issues. + +Hive-style partitioning also has some significant downsides. + +* It’s only suitable for low-cardinality columns. +* It can create many small files, especially if you use the wrong partition key or frequently update the Delta table. +* It can cause some queries that don’t rely on the partition key to run slower (because of the excessive number of small files). A large number of small files is problematic for I/O throughput. + +Hive-style partitioning can be a great data management tactic and a fantastic option for many Delta tables. Beware of the downsides before partitioning your tables. + +You can use Hive-style partitioning in conjunction with Z Ordering. You can partition a table by one column and Z Order by another. They’re different tactics that aim to help you skip more files and run queries faster. + +Let’s look at some of the built-in Delta features that help maintain the integrity of your tables. + +## Use appropriate quality controls + +Delta Lake supports schema enforcement and column constraints to protect the integrity of your data. + +Delta Lake enabled schema enforcement by default, so you can only append data to an existing table with the same exact schema. You can bypass schema enforcement by enabling schema evolution, which allows you to append mismatched schemas to a table. + +You should only enable schema evolution when you want to allow the schema of your table to change. You should not enable schema evolution if you don’t want this flexibility. Schema enforcement is a good default setting. + +Column-level constraints prevent you from appending data that fail SQL predicates. For example, you may add a constraint that requires all the values in the `age` column of a table to be positive. + +You should add column constraints to your table whenever you want a column only to include values that satisfy a predicate. + +No data is appended when you apply a constraint and a row check fails. For example, if you try to append 100 rows of data to a table and one row has a failing check, then no data is added. + +When you have column constraints, it’s often a good idea to append the failing data to a “quarantine table” and the passing data to the main Delta table. Or you can filter out the failing rows and just append the passing rows. Keeping a history of the failing rows in a quarantine table is helpful for debugging. + +See here to learn more about [Delta Lake constraints](https://delta-io.github.io/delta-rs/usage/constraints/). + +## Best practices for DML operations + +DML operations like deleting, updating, and merging write existing data in new files and mark existing files for deletion in the transaction log. Rewriting data files is expensive, so you want to minimize the number of rewritten files when you run DML operations. + +Delta Lake supports a table feature called deletion vectors that implements DML transactions more efficiently under the hood. Enabling deletion vectors is usually the best way to make DML operations run faster. Note: delta-rs doesn’t support deletion vectors yet. + +You should periodically purge deletion vectors because they can accumulate and slow subsequent read operations. Once you enable the feature, you must purge the deletion vectors in your table with an appropriate cadence. + +## Use vacuum to save storage costs + +Delta Lake supports transactions, which necessitates keeping old versions of data in storage, even the files marked for removal in the transactions log. + +Keeping old versions of Delta tables in storage is often desirable because it allows for versioned data, time travel, and rolling back tables to a previous state. + +If you don’t want to leverage older versions of a table, then you should remove the legacy files from storage with the vacuum command. Vacuum will remove all files older than the table retention period and marked for removal in the transaction log. + +You only need to vacuum when you perform operations that mark files for removal in the transaction log. An append-only table doesn’t create legacy files that need to be vacuumed. + +Create a good vacuum strategy for your tables to minimize your storage costs. + +## Delta Lake best practices to minimize costs + +Delta Lake helps you minimize costs in many ways: + +* It's a free, open source format (based on Parquet). It's not a proprietary format that you need to pay for. +* Delta tables store column-level min/max values in the transaction log, allowing file skipping. +* Delta tables can be optimized (small file compaction, Z Ordering, etc.), so your queries run faster. When your queries run faster, then you pay less on compute. +* Deletion vectors let you perform DML operations (delete, update, merge) much faster. If your delete operation runs 100x faster, then you pay 100x less compute. +* It's easy to remove legacy files from storage with VACUUM, which minimizes storage costs. + +You should understand your organization’s query patterns and use these features to minimize the overall cost. You need to assess tradeoffs. For example, Z Ordering is a computation that costs money, but it can save you lots of money in the long run if all your subsequent queries run a lot faster and use less compute. + +## Collect metadata stats on columns used for file skipping + +Delta tables don’t always store each column's min/max values. Some Delta Lake implementations only store min/max values for the first 32 columns in the table, for example. + +Delta Lake can only apply file-skipping when it has min/max values for the relevant columns stored in the transaction log. Suppose you’re running a filtering operation on `col_a,` for example. Delta Lake can only apply file skipping when the transaction log stores `col_a` min/max metadata. + +Ensure the transaction log stores metadata stats for all the columns that benefit from file skipping. + +## Don’t collect column metadata when it’s unnecessary + +It takes some time to compute column statistics when writing files, and it isn’t worth the effort if you cannot use the column for file skipping. + +Suppose you have a table column containing a long string of arbitrary text. It’s unlikely that this column would ever provide any data-skipping benefits. So, you can just avoid the overhead of collecting the statistics for this particular column. + +## Additional reading + +Delta Lake relies on transactions, and you should check out [this page to learn more](https://delta-io.github.io/delta-rs/how-delta-lake-works/delta-lake-acid-transactions/). + +Many Delta Lake performance benefits rely on [file skipping](https://delta-io.github.io/delta-rs/how-delta-lake-works/delta-lake-file-skipping/), which you should understand well to get the most out of Delta. + +## Conclusion + +Delta Lake is a powerful technology that makes your data pipelines more reliable, saves money, and makes everyday data processing tasks easy. + +You need to learn how Delta Lake works at a high level to leverage Delta's power fully. You will not be able to leverage Delta Lake’s full performance potential if your table has improperly sized files or if you’re not colocating data in the same files to maximize data skipping, for example. + +Luckily, there are only a few details that are important to learn. You don’t need to know the implementation details - just the essential high-level concepts. diff --git a/docs/delta-lake-big-data-small-data.md b/docs/delta-lake-big-data-small-data.md new file mode 100644 index 0000000000..060943549f --- /dev/null +++ b/docs/delta-lake-big-data-small-data.md @@ -0,0 +1,67 @@ +# Delta Lake for big data and small data + +Delta Lake is an excellent storage format for big data and small data. + +This post explains why Delta Lake is suitable for massive datasets and why many of these features that are great, even for tiny tables. Delta Lake is fine for a table with less than 1 GB of data or hundreds of petabytes of data. + +Let’s start by discussing the features that are great for small data. + +## Delta Lake for small data tables + +Delta Lake has many features that are useful for small datasets: + +* [Reliable transactions](https://delta-io.github.io/delta-rs/how-delta-lake-works/delta-lake-acid-transactions/) +* Better performance via file skipping +* DML operations to make deletes, updates, and merges easy and performant +* Features like schema enforcement and constraints to enforce data quality +* Versioned data & time travel + +All of these features are great for large and small tables. + +Delta Lake DML operations are ACID transactions, so they either finish entirely or don’t finish at all. Delta tables don’t require any downtime while DML operations are running. The Delta Lake user experience is better than a data lake that doesn’t support transactions and has downtime while running DML operations. + +The Delta Lake API also makes it easy to run DML operations. You can delete a line of code from a Delta table with a single line of code. Writing code to delete rows from CSV files is more challenging, especially if you want to implement this operation efficiently. + +Delta Lake has built-in checks to retain the integrity of your tables. For example, Delta tables have schema enforcement and prevent you from appending DataFrames with mismatched schema from the existing table. Delta Lake also lets you add constraints that only allow appending specific values to a column. Data quality is also essential for small tables! + +Delta Lake splits data into multiple files with file-level metadata in the transaction log, so query engines can sometimes skip data. Data skipping can be a huge performance benefit, depending on how much data can be ignored by the query engine. + +As previously mentioned, Delta tables record all DML operations as transactions. Recording operations as transactions means that existing data isn’t mutated. So Delta Lake provides versioned data and time travel out of the box. Versioning data is better because it allows you to roll back mistakes and compare the state of the table at different points in time. + +Delta Lake has many useful features for small data tables. Let’s look at how Delta Lake is scalable for massive datasets. + +## Delta Lake for large data tables + +Delta Lake is designed to be scalable and can handle tables with terabytes or petabytes of data. + +[See here](https://www.databricks.com/dataaisummit/session/flink-delta-driving-real-time-pipelines-doordash/) for an example of an organization ingesting 220 TB of data into a Delta table daily. + +Delta tables store data in Parquet files, and cloud object stores allow engines to write any number of files. Delta tables store metadata information in the transaction log as JSON files, which are periodically compacted into Parquet files, so an arbitrarily large amount of Delta table metadata can also be stored. + +Delta Lake transactions and concurrency protection maintain the integrity of tables, even for large write operations or long-running computations. + +It’s well known that Delta tables are scalable, even for the most enormous tables. + +## Small data operations on large tables + +Delta Lake is flexible and allows you to use “small data engines,” even for large tables, depending on the computation. + +Suppose you have a Delta table containing 10 TB of data and a pipeline that appends 0.5 GB of data to the table every hour. You don’t need a big data query engine to append a small amount of data. You can set up this job to run the Delta table append with a small data engine like pandas or Polars. + +Delta tables are flexible and interoperable with many technologies so that you can use the right tool for each data processing job. This allows you to design pipelines how you’d like and minimize costs. + +## When Delta Lake isn’t needed + +You don’t need Delta Lake for a small dataset that never changes and can be stored in a single Parquet file. + +Suppose you have a 0.5 GB dataset in a Parquet file that never needs to be updated. You can just keep that data in a Parquet table. Reading the metadata from the Parquet footer of a single file isn’t expensive. You won’t be taking advantage of Delta Lake's features like transactions, convenient DML operations, or versioned data. + +But in most cases, it’s best to use Delta Lake because its features protect the integrity of your tables and make your life easier. + +## Conclusion + +Delta Lake is well known for being scalable to huge tables but is also an excellent technology for small tables. + +Delta Lake is a lightweight technology, so there is little overhead. Writing the metadata file after performing a transaction is fast. It’s a minuscule cost, considering the benefits you receive. + +Many reasons that make Delta Lake better than data lakes for large tables also apply to small tables! diff --git a/docs/how-delta-lake-works/delta-lake-acid-transactions.md b/docs/how-delta-lake-works/delta-lake-acid-transactions.md new file mode 100644 index 0000000000..46dbd402e1 --- /dev/null +++ b/docs/how-delta-lake-works/delta-lake-acid-transactions.md @@ -0,0 +1,269 @@ +# Delta Lake Transactions + +This page teaches you about Delta Lake transactions and why transactions are important in production data settings. Data lakes don’t support transactions and this is a huge downside because they offer a poor user experience, lack functionality, and can easily be corrupted. + +Transactions on Delta Lake tables are operations that change the state of table and record descriptive entries (metadata) of those changes to the Delta Lake transaction log. Here are some examples of transactions: + +* Deleting rows +* Appending to the table +* Compacting small files +* Upserting +* Overwriting rows + +All Delta Lake write operations are transactions in Delta tables. Reads actually aren’t technically transactions because they don’t result in new entries being appended to the transaction log. + +## What are transactions? + +Transactions are any Delta operation that change the underlying files of a Delta table and result in new entries metadata entries in the transaction log. Some Delta operations rearrange data in the existing table (like Z Ordering the table or compacting the small files) and these are also transactions. Let’s look at a simple example. + +Suppose you have a Delta table with the following data: + +``` +num animal +1 cat +2 dog +3 snake +``` + +Here’s how to create this Delta table: + +```python +import pandas as pd +from deltalake import write_deltalake, DeltaTable + +df = pd.DataFrame({"num": [1, 2, 3], "animal": ["cat", "dog", "snake"]}) +write_deltalake("tmp/my-delta-table", df) +``` + +Here are the files created in storage. + +``` +tmp/my-delta-table +├── 0-fea2de92-861a-423e-9708-a9e91dafb27b-0.parquet +└── _delta_log + └── 00000000000000000000.json +``` + +Let’s perform an operation to delete every animal from the Delta table that is a cat. + +```python +dt = DeltaTable("tmp/my-delta-table") +dt.delete("animal = 'cat'") +``` + +Let’s take a look at the contents of the Delta table now that the transaction is complete: + +``` +tmp/my-delta-table +├── 0-fea2de92-861a-423e-9708-a9e91dafb27b-0.parquet +├── _delta_log +│ ├── 00000000000000000000.json +│ └── 00000000000000000001.json +└── part-00001-90312b96-b487-4a8f-9edc-1b9b3963f136-c000.snappy.parquet +``` + +Notice the `00000000000000000001.json` file that was added to the transaction log to record this transaction. Let’s inspect the content of the file. + +``` +{ + "add": { + "path": "part-00001-90312b96-b487-4a8f-9edc-1b9b3963f136-c000.snappy.parquet", + "partitionValues": {}, + "size": 858, + "modificationTime": 1705070631953, + "dataChange": true, + "stats": "{\"numRecords\":2,\"minValues\":{\"num\":2,\"animal\":\"dog\"},\"maxValues\":{\"num\":3,\"animal\":\"snake\"},\"nullCount\":{\"num\":0,\"animal\":0}}", + "tags": null, + "deletionVector": null, + "baseRowId": null, + "defaultRowCommitVersion": null, + "clusteringProvider": null + } +} +{ + "remove": { + "path": "0-fea2de92-861a-423e-9708-a9e91dafb27b-0.parquet", + "dataChange": true, + "deletionTimestamp": 1705070631953, + "extendedFileMetadata": true, + "partitionValues": {}, + "size": 895 + } +} +{ + "commitInfo": { + "timestamp": 1705070631953, + "operation": "DELETE", + "operationParameters": { + "predicate": "animal = 'cat'" + }, + "readVersion": 0, + "operationMetrics": { + "execution_time_ms": 8013, + "num_added_files": 1, + "num_copied_rows": 2, + "num_deleted_rows": 1, + "num_removed_files": 1, + "rewrite_time_ms": 2, + "scan_time_ms": 5601 + }, + "clientVersion": "delta-rs.0.17.0" + } +} +``` + +We can see that this transaction includes two components: + +* Remove file `0-fea2de92-861a-423e-9708-a9e91dafb27b-0.parquet` +* Add file `part-00001-90312b96-b487-4a8f-9edc-1b9b3963f136-c000.snappy.parquet` + +Transactions are recorded in the transaction log. The transaction log is also referred to as the table metadata and is the `_delta_log` directory in storage. + +Let’s see how Delta Lake implements transactions. + +## How Delta Lake implements transactions + +Here is how Delta Lake implements transactions: + +1. Read the existing metadata +2. Read the existing Parquet data files +3. Write the Parquet files for the current transaction +4. Record the new transaction in the transaction log (if there are no conflicts) + +Let’s recall our delete operation from the prior section and see how it fits into this transaction model: + +1. We read the existing metadata to find the file paths for the existing Parquet files +2. We read the existing Parquet files and identify the files that contains data that should be removed +3. We write new Parquet files with the deleted data filtered out +4. Once the new Parquet files are written, we check for conflicts and then make an entry in the transaction log. The next section will discuss transaction conflicts in more detail. + +Blind append operations can skip a few steps and are executed as follows: + +1. Write the Parquet files for the current transaction +2. Record the new transaction in the metadata + +Delta implements a non locking MVCC (multi version concurrency control) so writers optimistically write new data and simply abandon the transaction if it conflicts at the end. The alternative would be getting a lock at the start thereby guaranteeing the transaction immediately. + +Let’s look at the case when a Delta Lake transaction conflicts. + +## How Delta Lake transactions can conflict + +Suppose you have a transaction that deletes a row of data that’s stored in FileA (Transaction 1). While this job is running, there is another transaction that deletes some other rows in FileA (Transaction 2). Transaction 1 finishes running first and is recorded in the metadata. + +Before Transaction 2 is recorded as a transaction, it will check the metadata, find that Transaction 2 conflicts with a transaction that was already recorded (from Transaction 1), and error without recording a new transaction. + +Transactions 2 will write Parquet data files, but will not be recorded as a transaction, so the data files will be ignored. The zombie Parquet files can be easily cleaned up via subsequent vacuum operations. + +Transaction 2 must fail otherwise it would cause the data to be incorrect. + +Delta Lake transactions prevent users from making changes that would corrupt the table. Transaction conflict behavior can differ based on isolation level, which controls the degree to which a transaction must be isolated from modifications made by other concurrent transactions. More about this in the concurrency section. + +## Transactions rely on atomic primitives storage guarantees + +Suppose you have two transactions that are finishishing at the same exact time. Both of these transactions look at the existing Delta Lake transaction log, see that the latest transaction was `003.json` and determine that the next entry should be `004.json`. + +If both transactions are recorded in the `004.json` file, then one of them will be clobbered, and the transaction log entry for the clobbered metadata entry will be lost. + +Delta tables rely on storage systems that provide atomic primitives for safe concurrency. The storage system must allow Delta Lake to write the file, _only if it does not exist already_, and error out otherwise. The storage system must NOT permit concurrent writers to overwrite existing metadata entries. + +Some clouds have filesystems that don’t explicitly support these atomic primitives, and therefore must be coupled with other services to provide the necessary guarantees. + +## Delta Lake transactions are only for a single table + +Delta Lake transactions are only valid for a single table. + +Some databases offer transaction support for operations that impact multiple tables. Delta Lake does not support multi-table transactions. + +## Data lakes don’t support transactions + +Data lakes consist of many files in a storage system (e.g. a cloud storage system) and don’t support transactions. + +Data lakes don’t have a metadata layer, conflict resolution, or any way to store information about transactions. + +Data lakes are prone to multiple types of errors because they don’t support transactions: + +* Easy to corrupt +* Downtime/unstable state while jobs are running +* Operations can conflict + +Data lakes have many downsides and it’s almost always better to use a lakehouse storage system like Delta Lake compared to a data lake. + +## ACID Transactions + +We’ve already explored how Delta Lake supports transactions. This section explains how Delta Lake transactions have the Atomic, Consistent, Isolated and Durable (ACID transaction) properties. Reading this section is optional. + +ACID transactions are commonplace in databases but notably absent for data lakes. + +Delta Lake’s ACID transaction support is one of the major reasons it is almost always a better option than a data lake. + +Let’s explore how Delta Lake allows for ACID transactions. + +**Atomic transactions** + +An atomic transaction either fully completes or fully fails, with nothing in between. + +Delta Lake transactions are atomic, unlike data lake transactions that are not atomic. + +Suppose you have a job that’s writing 100 files to a table. Further suppose that the job errors out and the cluster dies after writing 40 files: + +* For a Delta table, no additional data will be added to the table. Parquet files were written to the table, but the job errored, so no transaction log entry was added and no data was added to the table. +* For a data lake, the 40 files are added and the transaction “partially succeeds”. + +For data tables, it’s almost always preferable to have a transaction that “fully fails” instead of one that “partially succeeds” because partial writes are hard to unwind and debug. + +Delta Lake implements atomic transactions by writing data files first before making a new entry in the Delta transaction log. + +These guarantees are provided at the protocol level through the "transaction" abstraction. We’ve already discussed what constitutes a transaction for Delta Lake. + +If there is an error with the transaction and some files don’t get written, then no metadata entry is made and the partial data write is ignored. The zombie Parquet files can be easily cleaned up via subsequent vacuum operations. + +Now let’s look at how Delta Lake also provides consistent transactions. + +**Consistent transactions** + +Consistency means that transactions won’t violate integrity constraints on the Delta table. + +Delta Lake has two types of consistency checks: + +* Schema enforcement checks +* Column constraints + +Schema enforcement checks verify that new data appended to a Delta table matches the schema of the existing table. You cannot append data with a different schema, unless you enable schema evolution. + +Delta Lake column constraints allow users to specify the requirements of data that’s added to a Delta table. For example, if you have an age column with a constraint that requires the value to be positive, then Delta Lake will reject appends of any data that doesn’t meet the constraint. + +Data lakes don’t support schema enforcement or column constraints. That’s another reason why data lakes are not ACID-compliant. + +**Isolated transactions** + +Isolation means that transactions are applied to a Delta table sequentially. + +Delta Lake transactions are persisted in monotonically increasing transaction files, as we saw in the previous example. First `00000000000000000000.json`, then `00000000000000000001.json`, then `00000000000000000002.json`, and so on. + +Delta Lake uses concurrency control to ensure that transactions are executed sequentially, even when user operations are performed concurrently. The next page of this guide explains concurrency in Delta Lake in detail. + +**Durable transactions** + +Delta tables are generally persisted in cloud object stores which provide durability guarantees. + +Durability means that all transactions that are successfully completed will always remain persisted, even if there are service outages or program crashes. + +Suppose you have a Delta table that’s persisted in Azure blob storage. The Delta table transactions that are committed will always remain available, even in these circumstances: + +* When there are Azure service outages +* If a computation cluster that’s writing the Delta table crashes for some reason +* Two operations are running concurrently and one of them fails + +Successful transactions are always registered in the Delta table and persisted no matter what. + +## Conclusion + +Delta Lake supports transactions which provide necessary reliability guarantees for production data systems. + +Vanilla data lakes don’t provide transactions and this can cause nasty bugs and a bad user experience. Let’s look at a couple of scenarios when the lack of transactions cause a poor user experience: + +* While running a compaction operation on a data lake, newly compacted “right sized” files are added before the small files are deleted. If you read the data lake while this operation is running, you will see duplicate data. +* While writing to a data lake, a job might fail, which leaves behind partially written files. These files are corrupt, which means that the data lake cannot be read until the corrupt files are manually removed. +* Users want to run a simple DML operation like deleting a few rows of data which require a few files to be rewritten. This operation renders the data lake unusable until it’s done running. + +Transactions are a key advantage of Delta Lake vs. data lakes. There are many other advantages, but proper transactions are necessary in production data environments. diff --git a/docs/how-delta-lake-works/delta-lake-file-skipping.md b/docs/how-delta-lake-works/delta-lake-file-skipping.md new file mode 100644 index 0000000000..f6562e3952 --- /dev/null +++ b/docs/how-delta-lake-works/delta-lake-file-skipping.md @@ -0,0 +1,183 @@ +# Delta Lake File Skipping + +Delta tables store file-level metadata information, which allows for a powerful optimization called file skipping. + +This page explains how Delta Lake implements file skipping, how to optimize your tables to maximize file skipping, and the benefits of file skipping. + +Let’s start by looking at the file-level metadata in Delta tables. + +## Delta Lake file metadata + +Delta Lake stores metadata about each file's min/max values in the table. Query engines can skip entire files when they don’t contain data that’s relevant to the query. + +Suppose you have a Delta table with data stored in two files and has the following metadata. + +``` +filename min_name max_name min_age max_age +fileA alice joy 12 46 +fileB allan linda 34 78 +``` + +Suppose you want to run the following query: `select * from the_table where age < 20`. + +The engine only needs to read `fileA` to execute this query. `fileB` has a `min_age` of 34, so we know there aren’t any rows of data with an `age` less than 20. + +The benefit of file skipping depends on the query and the data layout of the Delta table. Some queries cannot take advantage of any file skipping. Here’s an example query that does not benefit from file skipping: `select * from the_table group by age`. + +Let’s recreate this example with Polars to drive the point home. + +Start by writing out one file of data: + +```python +import polars as pl +from deltalake import DeltaTable + +df = pl.DataFrame({"name": ["alice", "cat", "joy"], "age": [12, 35, 46]}) +df.write_delta("tmp/a_table") +``` + +Now, write out another file of data: + +```python +df = pl.DataFrame({"name": ["allan", "brian", "linda"], "age": [34, 35, 78]}) +df.write_delta("tmp/a_table", mode="append") +``` + +Here are the contents of the Delta table: + +``` +tmp/a_table +├── 0-7d414a88-a634-4c2f-9c5b-c29b6ee5f524-0.parquet +├── 1-0617ef60-b17b-46a5-9b0f-c7dda1b73eee-0.parquet +└── _delta_log + ├── 00000000000000000000.json + └── 00000000000000000001.json +``` + +Now run a query to fetch all the records where the age is less than 20: + +```python +pl.scan_delta("tmp/a_table").filter(pl.col("age") < 20).collect() +``` + +``` ++-------+-----+ +| name | age | +| --- | --- | +| str | i64 | ++=============+ +| alice | 12 | ++-------+-----+ +``` + +Polars can use the Delta table metadata to skip the file that does not contain data relevant to the query. + +## How Delta Lake implements file skipping + +Here’s how engines execute queries on Delta tables: + +* Start by reading the transaction log to get the file paths, file sizes, and min/max value for each column +* Parse the query and push down the predicates to skip files +* Read the minimal subset of the files needed for the query + +Some file formats don’t allow for file skipping. For example, CSV files don’t have file-level metadata, so query engines can’t read a minimal subset of the data. The query engine has to check all the files, even if they don’t contain any relevant data. + +When data is in Parquet files, the query engine can open up all the files, read the footers, build the file-level metadata, and perform file skipping. Fetching metadata in each file is slower than grabbing the pre-built file-level metadata from the transaction log. + +Now, let’s see how to structure your tables to allow for more file skipping. + +## File skipping for different file sizes + +Delta tables store data in files. Smaller files allow for more file skipping compared to bigger files. + +However, an excessive number of small files isn’t good because it creates I/O overhead and slows down queries. + +Your Delta tables should have files that are “right-sized”. For a table with 150 GB of data, 5 GB files would probably be too large, and 10 KB files would be too small. It’s generally best to store data in files that are between 100 MB and 1 GB. + +Delta Lake has [an optimize function](https://delta-io.github.io/delta-rs/usage/optimize/small-file-compaction-with-optimize/) that performs small file compaction, so you don’t need to program this logic yourself. + +Now, let's investigate how to store data in files to maximize the file skipping opportunities. + +## How to maximize file skipping + +You can maximize file-skipping by colocating similar data in the same files. + +Suppose you have a table with test scores and frequently run queries that filter based on the `test_score` column. + +``` +filename min_test_score max_test_score +fileA 45 100 +fileB 65 98 +fileC 50 96 +``` + +Suppose you want to run the following query: `select * from exams where test_score > 90`. + +This query cannot skip files, given the current organization of the data. You can rearrange the data to colocate similar test scores in the same files to allow for file skipping. Here’s the new layout: + +``` +filename min_test_score max_test_score +fileD 45 70 +fileE 55 80 +fileF 78 100 +``` + +The query (`select * from exams where test_score > 90`) can skip two of the three files with the new Delta table layout. The query engine only has to read `fileF` for this query. + +Now, let’s look at how file skipping works with string values. + +## How file skipping works with strings + +File skipping is also effective when filtering on string values. + +Suppose you have a table with `person_name` and `country` columns. There are millions of rows of data. Here are the first three rows of data: + +``` +person_name country +person1 angola +person2 china +person3 mexico +``` + +The Delta table contains three files with the following metadata: + +``` +filename min_country max_country +fileA albania mali +fileB libia paraguay +fileC oman zimbabwe +``` + +Suppose you want to run the following query: `select * from some_people where country = 'austria'`. + +You only need to read the data in `fileA` to run this query. The `min_country` value for `fileB` and `fileC` are greater than “austria”, so we know those files don’t contain any data relevant to the query. + +File skipping can also be a robust optimization for string values. Now, let’s see how file skipping works for partitioned tables. + +## File skipping for partitioned tables + +You can partition Delta tables for file skipping as well. Suppose we have the same data as in the previous section, but the table is partitioned by `country`. + +Here’s the Delta table: + +``` +filename partition +fileA albania +fileB libia +fileC oman +fileD jamaica +fileE albania +fileF oman +``` + +Suppose you want to run the following query on this partitioned table: `select * from some_partitioned_table where country = 'albania'`. + +You only need to read `fileA` and `fileE` to execute this query. Delta Lake provides the file-level partition metadata in the transaction log so that this query will run quickly. + +## Conclusion + +Delta Lake allows for file skipping, which is a powerful performance optimization. + +Delta Lake also provides built-in utilities to colocate data in the same files like partitioning, Z Ordering, and compaction to improve file skipping. + +Delta Lake users need to know how to assess the tradeoffs of these techniques to optimize file skipping. Users also need to understand the most frequent query patterns of their tables to best allow for file maximal file skipping. diff --git a/docs/index.md b/docs/index.md index 8dfa67f924..99b7dc6cb3 100644 --- a/docs/index.md +++ b/docs/index.md @@ -1,6 +1,6 @@ # The deltalake package -This is the documentation for the native Rust/Python implementation of Delta Lake. It is based on the delta-rs Rust library and requires no Spark or JVM dependencies. For the PySpark implementation, see [delta-spark](https://docs.delta.io/latest/api/python/index.html) instead. +This is the documentation for the native Rust/Python implementation of Delta Lake. It is based on the delta-rs Rust library and requires no Spark or JVM dependencies. For the PySpark implementation, see [delta-spark](https://docs.delta.io/latest/api/python/spark/index.html) instead. This module provides the capability to read, write, and manage [Delta Lake](https://delta.io/) tables with Python or Rust without Spark or Java. It uses [Apache Arrow](https://arrow.apache.org/) under the hood, so is compatible with other Arrow-native or integrated libraries such as [pandas](https://pandas.pydata.org/), [DuckDB](https://duckdb.org/), and [Polars](https://www.pola.rs/). diff --git a/docs/integrations/delta-lake-arrow.md b/docs/integrations/delta-lake-arrow.md index 6da4d5fcc2..70965e3b74 100644 --- a/docs/integrations/delta-lake-arrow.md +++ b/docs/integrations/delta-lake-arrow.md @@ -2,7 +2,7 @@ Delta Lake tables can be exposed as Arrow tables and Arrow datasets, which allows for interoperability with a variety of query engines. -This page shows you how to convert Delta tables to Arrow data structures and teaches you the difference between Arrow tables and Arrow datasets. +This page shows you how to convert Delta tables to Arrow data structures and teaches you the difference between Arrow tables and Arrow datasets. Tables are "eager" and datasets are "lazy", which has important performance implications, keep reading to learn more! ## Delta Lake to Arrow Dataset diff --git a/docs/integrations/delta-lake-dagster.md b/docs/integrations/delta-lake-dagster.md new file mode 100644 index 0000000000..0fe413c7a0 --- /dev/null +++ b/docs/integrations/delta-lake-dagster.md @@ -0,0 +1,278 @@ +# Using Delta Lake with Dagster¶ + +Delta Lake is a great storage format for Dagster workflows. This page will explain why and how to use Delta Lake with Dagster. + +You will learn how to use the Delta Lake I/O Manager to read and write your Dagster Software-Defined Assets (SDAs). You will also learn about the unique advantages Delta Lake offers the Dagster community. + +Here are some of the benefits that Delta Lake provides Dagster users: +- native PyArrow integration for lazy computation of large datasets, +- more efficient querying with file skipping via Z Ordering and liquid clustering +- built-in vacuuming to remove unnecessary files and versions +- ACID transactions for reliable writes +- smooth versioning integration so that versions can be use to trigger downstream updates. +- surfacing table stats based on the file statistics + + +## Dagster I/O Managers +Dagster uses [I/O Managers](https://docs.dagster.io/concepts/io-management/io-managers#overview) to simplify data reads and writes. I/O Managers help you reduce boilerplate code by storing Dagster Asset and Op outputs and loading them as inputs to downstream objects. They make it easy to change where and how your data is stored. + +You only need to define your I/O Manager and its settings (such as storage location and schema) once and the I/O Manager will take care of correctly reading and writing all your Dagster Assets automatically. + +If you need lower-level access than the Dagster I/O Managers provide, take a look at the Delta Table Resource. + +## The Delta Lake I/O Manager +You can easily read and write Delta Lake Tables from Dagster by using the `DeltaLakeIOManager()`. + +Install the DeltaLakeIOManager: + +``` +pip install dagster-deltalake +``` + +Next, configure the following settings in your project’s `__init__.py` file: +- `io_manager`: set this to `DeltaLakeIOManager()`, this sets the default I/O Manager for all your Assets + +Within the DeltaLakeIOManager, define: +- `root_uri`: the root path where your Delta Tables will be created +- `storage_options`: configuration for accessing storage location +- `schema`: name of schema to use (optional, defaults to public) + +``` +defs = Definitions( + assets=all_assets, + resources={ + "io_manager": DeltaLakePyarrowIOManager( + root_uri="path/to/deltalake", + storage_options=LocalConfig(), + schema="dagster_deltalake", + ), + }, +) +``` + +Now, when you materialize an Asset, it will be saved as a Delta Lake in a folder `dagster_deltalake/asset_name` under the root directory `path/to/deltalake`. + +The default Delta Lake I/O Manager supports Arrow reads and writes. You can also use the Delta Lake I/O Manager with [pandas](#using-delta-lake-and-dagster-with-pandas) or [polars](#using-delta-lake-and-dagster-with-polars). + +## Creating Delta Lake Tables with Dagster +You don’t need to do anything else to store your Dagster Assets as Delta Lake tables. The I/O Manager will handle storing and loading your Assets as Delta Lake tables from now on. + +You can proceed to write Dagster code as you normally would. For example, you can create an Asset that reads in some toy data about animals and writes it out to an Arrow Table: + +``` +import pyarrow as pa +from pyarrow import csv + +from dagster import asset + +@asset +def raw_dataset() -> pa.Table: + n_legs = pa.array([2, 4, None, 100]) + animals = pa.array(["Flamingo", "Horse", "Brittle stars", "Centipede"]) + data = {'n_legs': n_legs, 'animals': animals} + + return pa.Table.from_pydict(data) +``` + +When you materialize the Asset defined above (using the config settings defined earlier), the Delta Lake I/O Manager will create the table `dagster_deltalake/iris_dataset` if it doesn’t exist yet. + +### Overwrites when Rematerializing Assets +If the table does already exist at the specified location, the Delta Lake I/O Manager will perform an overwrite. Delta Lake’s transaction log maintains a record of all changes to your Delta Lake tables. You can inspect the record of changes to your Delta Lake tables by taking a look at these transaction logs. + +## Loading Delta Lake Tables in Downstream Assets +You can use Assets stored as Delta Lake tables as input to downstream Assets. Dagster and the Delta Lake I/O Manager make this easy for you. + +You can write Dagster code as you normally would. Pass the upstream Asset as an argument to the downstream object to set up the dependency. Make sure to define the correct data type. + +The Delta Lake I/O Manager will handle reading and writing the data from your Delta Lake. + +``` +import pyarrow as pa +from dagster import asset + +# ... raw_dataset asset is defined here ... + +@asset +def clean_dataset(raw_dataset: pa.Table) -> pa.Table: + return raw_dataset.drop_null() +``` + +## Reading Existing Delta Lake Tables into Dagster +You can make existing Delta Lake tables (that were not created in Dagster) available to your Dagster assets. Use the `SourceAsset` object and pass the table name as the key argument: + +``` +from dagster import SourceAsset + +iris_harvest_data = SourceAsset(key="more_animal_data") +``` + +This will load a table `more_animal_data` located at `/` as configured in the Definitions object above (see [Delta Lake I/O Manager](#the-delta-lake-io-manager) section). + +## Column Pruning +You can often improve the efficiency of your computations by only loading specific columns of your Delta table. This is called column pruning. + +With the Delta Lake I/O manager, you can select specific columns to load defining the `columns` in the `metadata` parameter of the `AssetIn` that loads the upstream Asset: + +``` +import pyarrow as pa +from dagster import AssetIn, asset + +# this example uses the clean_dataset Asset defined earlier + +@asset( + ins={ + "mammal_bool": AssetIn( + key="clean_dataset", + metadata={"columns": ["is_mammal", "animals"]}, + ) + } +) +def mammal_data(mammal_bool: pa.Table) -> pa.Table: + mammals = mammal_bool["is_mammal"].cast("bool") + animals = mammal_bool["animals"] + data = {"mammal_bool": mammals, "animals": animals} + return pa.Table.from_pydict(data) +``` + +Here, we select only the `sepal_length_cm` and `sepal_width_cm` columns from the `iris_dataset` table and load them into an `AssetIn` object called `iris_sepal`. This AssetIn object is used to create a new Asset `sepal_data`, containing only the selected columns. + +## Working with Partitioned Assets +Partitioning is an important feature of Delta Lake that can make your computations more efficient. The Delta Lake I/O manager helps you read and write partitioned data easily. You can work with static partitions, time-based partitions, multi-partitions, and dynamic partitions. + +For example, you can partition the Iris dataset on the `species` column as follows: + +``` +import pyarrow as pa + +from dagster import StaticPartitionsDefinition, asset + +@asset( + partitions_def=StaticPartitionsDefinition( + ["Human", "Horse",] + ), + metadata={"partition_expr": "n_legs"}, +) +def dataset_partitioned( + context, + clean_dataset: pa.Table, + ) -> pa.Table: + animals = context.asset_partition_key_for_output() + table = clean_dataset + + return table.filter(pc.field("animals") == animals) +``` + +To partition your data, make sure to include the relevant `partitions_def` and `metadata` arguments to the `@asset` decorator. Refer to the Dagster documentation on [partitioning assets](https://docs.dagster.io/concepts/partitions-schedules-sensors/partitioning-assets) for more information. + +## Using Delta Lake and Dagster with Pandas +To read and write data to Delta Lake using pandas, use the `DeltaLakePandasIOManager()`. + +You will need to install it using: + +``` +pip install dagster-deltalake-pandas +``` + +In your `Definitions` object, change the `io_manager` to `DeltaLakePandasIOManager()`: + +``` +from dagster_deltalake_pandas import DeltaLakePandasIOManager + + +defs = Definitions( + assets=all_assets, + resources={ + "io_manager": DeltaLakePandasIOManager( + root_uri="path/to/deltalake", + storage_options=LocalConfig(), + schema="dagster_deltalake", + ), + }, +) +``` + +Now you can read and write Dagster Assets defined as pandas DataFrames in Delta Lake format. For example: + +``` +import pandas as pd +from dagster import asset + +@asset +def iris_dataset() -> pd.DataFrame: + return pd.read_csv( + "https://docs.dagster.io/assets/iris.csv", + names=[ + "sepal_length_cm", + "sepal_width_cm", + "petal_length_cm", + "petal_width_cm", + "species", + ], + ) +``` + +## Using Delta Lake and Dagster with Polars +To read and write data to Delta Lake using pandas, use the `DeltaLakePolarsIOManager()`. + +You will need to install it using: + +``` +pip install dagster-deltalake-polars +``` + +In your `Definitions` object, change the `io_manager` to `DeltaLakePolarsIOManager()`: + +``` +from dagster_polars import DeltaLakePolarsIOManager + +defs = Definitions( + assets=all_assets, + resources={ + "io_manager": DeltaLakePolarsIOManager( + root_uri="path/to/deltalake", + storage_options=LocalConfig(), + schema="dagster_deltalake", + ), + }, +) +``` + +Now you can read and write Dagster Assets defined as Polars DataFrames in Delta Lake format. For example: + +``` +import polars as pl +from dagster import asset + + +@asset +def iris_dataset() -> pl.DataFrame: + return pl.read_csv( + "https://docs.dagster.io/assets/iris.csv", + new_columns=[ + "sepal_length_cm", + "sepal_width_cm", + "petal_length_cm", + "petal_width_cm", + "species", + ], + has_header=False +) +``` + +## Delta Lake Table Resource +I/O managers are a helpful tool in many common usage situations. But when you need lower-level access, the I/O Manager might not be the right tool to use. In these cases you may want to use the Delta Lake Table Resource. + +The Delta Lake Table Resource is a low-level access method to the table object. It gives you more fine-grained control and allows for modeling of more complex data. You can also use the Table Resource to run optimization and vacuuming jobs. + +## Schema and Constraint Enforcement +Delta Lake provides built-in checks to ensure schema consistency when appending data to a table, as well as the ability to evolve the schema. This is a great feature for the Dagster community as it prevents bad data from being appended to tables, ensuring data consistency and accuracy. + +Read more about how to add constraints to a table in [the Delta Lake documentation](https://delta-io.github.io/delta-rs/usage/constraints/). + +## Z-Ordering +Delta Lake offers Z-ordering functionality to colocate similar data in the same files. This can make your Delta Table queries much more efficient via file skipping. Dagster users can now benefit from this great feature through the Delta Lake I/O Manager. + +Read more about Z-Ordering on [the Delta Lake blog](https://delta.io/blog/2023-06-03-delta-lake-z-order/). + +## Contribute +To contribute to the Delta Lake and Dagster integration, go to [link] diff --git a/docs/integrations/delta-lake-dask.md b/docs/integrations/delta-lake-dask.md new file mode 100644 index 0000000000..cb3cab27f7 --- /dev/null +++ b/docs/integrations/delta-lake-dask.md @@ -0,0 +1,182 @@ +# Using Delta Lake with Dask + +Delta Lake is a great storage format for Dask analyses. This page explains why and how to use Delta Lake with Dask. + +You will learn how to read Delta Lakes into Dask DataFrames, how to query Delta tables with Dask, and the unique advantages Delta Lake offers the Dask community. + +Here are some of the benefits that Delta Lake provides Dask users: +- better performance with file skipping +- enhanced file skipping via Z Ordering +- ACID transactions for reliable writes +- easy time-travel functionality + +> ❗️ dask-deltatable currently works with deltalake<=13.0. See https://github.com/dask-contrib/dask-deltatable/issues/65 + +## Install Dask-Deltatable + +To use Delta Lake with Dask, first install the library using + +``` +pip install dask-deltatable +``` + + +## Reading Delta Tables into a Dask DataFrame + +You can read data stored in a Delta Lake into a Dask DataFrame using `dask-deltatable.read_deltalake`. + +Let's read in a toy dataset to see what we can do with Delta Lake and Dask. You can access the data stored as a Delta Lake [on Github](https://github.com/rrpelgrim/delta-examples/tree/master/data) + +``` +import dask_deltatable as ddt + +# read delta table into Dask DataFrame +delta_path = "path/to/data/people_countries_delta_dask" +ddf = ddt.read_deltalake(delta_path) + +``` + +Dask is a library for efficient distributed computing and works with lazy evaluation. Function calls to `dask.dataframe` build a task graph in the background. To trigger computation, call `.compute()`: + +``` +> ddf.compute() + +| | first_name | last_name | country | continent | +|---:|:-------------|:------------|:----------|:------------| +| 0 | Ernesto | Guevara | Argentina | NaN | +| 0 | Bruce | Lee | China | Asia | +| 1 | Jack | Ma | China | Asia | +| 0 | Wolfgang | Manche | Germany | NaN | +| 1 | Soraya | Jala | Germany | NaN | +``` + + +You can read in specific versions of Delta tables by specifying a `version` number or a timestamp: + +``` +# with specific version +ddf = ddt.read_deltalake(delta_path, version=3) + +# with specific datetime +ddt.read_deltalake(delta_path, datetime="2018-12-19T16:39:57-08:00") +``` + +`dask-deltatable` also supports reading from remote sources like S3 with: + +``` +ddt.read_deltalake("s3://bucket_name/delta_path", version=3) +``` + +> To read data from remote sources you'll need to make sure the credentials are properly configured in environment variables or config files. Refer to your cloud provider documentation to configure these. + +## What can I do with a Dask Deltatable? + +Reading a Delta Lake in with `dask-deltatable` returns a regular Dask DataFrame. You can perform [all the regular Dask operations](https://docs.dask.org/en/stable/dataframe.html) on this DataFrame. + +Let's take a look at the first few rows: + +``` +> ddf.head(n=3) + +| | first_name | last_name | country | continent | +|---:|:-------------|:------------|:----------|------------:| +| 0 | Ernesto | Guevara | Argentina | nan | +``` + +`dask.dataframe.head()` shows you the first rows of the first partition in the dataframe. In this case, the first partition only has 1 row. + +This is because the Delta Lake has been partitioned by country: + +``` +> !ls ../../data/people_countries_delta_dask +_delta_log country=Argentina country=China country=Germany +``` + +`dask-deltatable` neatly reads in the partitioned Delta Lake into corresponding Dask DataFrame partitions: + +``` +> # see number of partitions +> ddf.npartitions +3 +``` + +You can inspect a single partition using `dask.dataframe.get_partition()`: + +``` +> ddf.get_partition(n=1).compute() + +| | first_name | last_name | country | continent | +|---:|:-------------|:------------|:----------|:------------| +| 0 | Bruce | Lee | China | Asia | +| 1 | Jack | Ma | China | Asia | +``` + +## Perform Dask Operations + +Let's perform some basic computations over the Delta Lake data that's now stored in our Dask DataFrame. + +Suppose you want to group the dataset by the `country` column: + +``` +> ddf.groupby(['country']).count().compute() + +| country | first_name | last_name | continent | +|:----------|-------------:|------------:|------------:| +| Argentina | 1 | 1 | 1 | +| China | 2 | 2 | 2 | +| Germany | 2 | 2 | 2 | +``` + +Dask executes this `groupby` operation in parallel across all available cores. + +## Map Functions over Partitions + +You can also use Dask's `map_partitions` method to map a custom Python function over all the partitions. + +Let's write a function that will replace the missing `continent` values with the right continent names. + +``` +# define custom python function + +# get na_string +df = ddf.get_partition(0).compute() +na_string = df.iloc[0].continent +na_string + +# define function +def replace_proper(partition, na_string): + if [partition.country == "Argentina"]: + partition.loc[partition.country=="Argentina"] = partition.loc[partition.country=="Argentina"].replace(na_string, "South America") + if [partition.country == "Germany"]: + partition.loc[partition.country=="Germany"] = partition.loc[partition.country=="Germany"].replace(na_string, "Europe") + else: + pass + return partition +``` + +Now map this over all partitions in the Dask DataFrame: + +``` +# define metadata and map function over partitions +> meta = dict(ddf.dtypes) +> ddf3 = ddf.map_partitions(replace_proper, na_string, meta=meta) +> ddf3.compute() + +| | first_name | last_name | country | continent | +|---:|:-------------|:------------|:----------|:--------------| +| 0 | Ernesto | Guevara | Argentina | South America | +| 0 | Bruce | Lee | China | Asia | +| 1 | Jack | Ma | China | Asia | +| 0 | Wolfgang | Manche | Germany | Europe | +| 1 | Soraya | Jala | Germany | Europe | +``` + +## Write to Delta Lake +After doing your data processing in Dask, you can write the data back out to Delta Lake using `to_deltalake`: + +``` +ddt.to_deltalake(ddf, "tmp/test_write") +``` + +## Contribute to `dask-deltalake` +To contribute, go to the [`dask-deltalake` Github repository](https://github.com/rrpelgrim/dask-deltatable). \ No newline at end of file diff --git a/docs/integrations/delta-lake-datafusion.md b/docs/integrations/delta-lake-datafusion.md new file mode 100644 index 0000000000..a9c2a3cae0 --- /dev/null +++ b/docs/integrations/delta-lake-datafusion.md @@ -0,0 +1,85 @@ +# Using Delta Lake with DataFusion + +This page explains how to use Delta Lake with DataFusion. + +Delta Lake offers DataFusion users better performance and more features compared to other formats like CSV or Parquet. + +Delta Lake works well with the DataFusion Rust API and the DataFusion Python API. It's a great option for all DataFusion users. + +Delta Lake also depends on DataFusion to implement SQL-related functionality under the hood. We will also discuss this dependency at the end of this guide in case you're interested in learning more about the symbiotic relationship between the two libraries. + +## Delta Lake performance benefits for DataFusion users + +Let's run some DataFusion queries on a Parquet file and a Delta table with the same data to learn more about the performance benefits of Delta Lake. + +Suppose you have the following dataset with 1 billion rows and 9 columns. Here are the first three rows of data: + +``` ++-------+-------+--------------+-------+-------+--------+------+------+---------+ +| id1 | id2 | id3 | id4 | id5 | id6 | v1 | v2 | v3 | +|-------+-------+--------------+-------+-------+--------+------+------+---------| +| id016 | id046 | id0000109363 | 88 | 13 | 146094 | 4 | 6 | 18.8377 | +| id039 | id087 | id0000466766 | 14 | 30 | 111330 | 4 | 14 | 46.7973 | +| id047 | id098 | id0000307804 | 85 | 23 | 187639 | 3 | 5 | 47.5773 | ++-------+-------+--------------+-------+-------+--------+------+------+---------+ +``` + +Here's how to register a Delta Lake table as a PyArrow dataset: + +```python +from datafusion import SessionContext +from deltalake import DeltaTable + +ctx = SessionContext() +table = DeltaTable("G1_1e9_1e2_0_0") +ctx.register_dataset("my_delta_table", table.to_pyarrow_dataset()) +``` + +Now query the table: + +```python +ctx.sql("select id1, sum(v1) as v1 from my_delta_table where id1='id096' group by id1") +``` + +That query takes 2.8 seconds to execute. + +Let's register the same dataset as a Parquet table, run the same query, and compare the runtime difference. + +Register the Parquet table and run the query: + +```python +path = "G1_1e9_1e2_0_0.parquet" +ctx.register_parquet("my_parquet_table", path) +ctx.sql("select id1, sum(v1) as v1 from my_parquet_table where id1='id096' group by id1") +``` + +This query takes 5.3 seconds to run. + +Parquet stores data in row groups and DataFusion can intelligently skip row groups that don't contain relevant data, so the query is faster than a file format like CSV which doesn't support row group skipping. + +Delta Lake stores file-level metadata information in the transaction log, so it can skip entire files when queries are executed. Delta Lake can skip entire files and then skip row groups within the individual files. This makes Delta Lake even faster than Parquet files, especially for larger datasets spread across many files. + +## Delta Lake features for DataFusion users + +Delta Lake also provides other features that are useful for DataFusion users like ACID transactions, concurrency protection, time travel, versioned data, and more. + +## Why Delta Lake depends on DataFusion + +Delta Lake depends on DataFusion to provide some end-user features. + +DataFusion is useful in providing SQL-related Delta Lake features. Some examples: + +* Update and merge are written in terms of SQL expressions. +* Invariants and constraints are written in terms of SQL expressions. + +Anytime we have to evaluate SQL, we need some sort of SQL engine. We use DataFusion for that. + +## Conclusion + +Delta Lake is a great file format for DataFusion users. + +Delta Lake also uses DataFusion to provide some end-user features. + +DataFusion and Delta Lake have a wonderful symbiotic relationship and play very nicely with each other. + +See [this guide for more information on Delta Lake and PyArrow](https://delta-io.github.io/delta-rs/integrations/delta-lake-arrow/) and why PyArrow Datasets are often a better option than PyArrow tables. diff --git a/docs/integrations/delta-lake-pandas.md b/docs/integrations/delta-lake-pandas.md index b14c1bd45b..ca60362838 100644 --- a/docs/integrations/delta-lake-pandas.md +++ b/docs/integrations/delta-lake-pandas.md @@ -250,10 +250,10 @@ Schema enforcement protects your table from getting corrupted by appending data ## Overwriting schema of table -You can overwrite the table contents and schema by setting the `overwrite_schema` option. Here's how to overwrite the table contents: +You can overwrite the table contents and schema by setting the `schema_mode` option. Here's how to overwrite the table contents: ```python -write_deltalake("tmp/some-table", df, mode="overwrite", overwrite_schema=True) +write_deltalake("tmp/some-table", df, mode="overwrite", schema_mode="overwrite") ``` Here are the contents of the table after the values and schema have been overwritten: @@ -267,6 +267,8 @@ Here are the contents of the table after the values and schema have been overwri +-------+----------+ ``` +If you want the schema to be merged instead, specify schema_mode="merge". + ## In-memory vs. in-storage data changes It's important to distinguish between data stored in-memory and data stored on disk when understanding the functionality offered by Delta Lake. diff --git a/docs/integrations/delta-lake-polars.md b/docs/integrations/delta-lake-polars.md index 0e029847e8..eec9a7c2f4 100644 --- a/docs/integrations/delta-lake-polars.md +++ b/docs/integrations/delta-lake-polars.md @@ -132,7 +132,7 @@ print(pl.read_delta("tmp/bear_delta_lake", version=1)) +-----+ ``` -Read the Delta table wihout specifying a version and see how it reads the latest version by default: +Read the Delta table without specifying a version and see how it reads the latest version by default: ``` print(pl.read_delta("tmp/bear_delta_lake")) diff --git a/docs/src/python/check_constraints.py b/docs/src/python/check_constraints.py new file mode 100644 index 0000000000..1bfa62d970 --- /dev/null +++ b/docs/src/python/check_constraints.py @@ -0,0 +1,24 @@ +def add_constraint(): + # --8<-- [start:add_constraint] + from deltalake import DeltaTable + + dt = DeltaTable("../rust/tests/data/simple_table") + + # Check the schema before hand + print(dt.schema()) + # Add the constraint to the table. + dt.alter.add_constraint({"id_gt_0": "id > 0"}) + # --8<-- [end:add_constraint] + + +def add_data(): + # --8<-- [start:add_data] + from deltalake import write_deltalake, DeltaTable + import pandas as pd + + dt = DeltaTable("../rust/tests/data/simple_table") + + df = pd.DataFrame({"id": [-1]}) + write_deltalake(dt, df, mode="append", engine="rust") + # _internal.DeltaProtocolError: Invariant violations: ["Check or Invariant (id > 0) violated by value in row: [-1]"] + # --8<-- [end:add_data] diff --git a/docs/src/python/operations.py b/docs/src/python/operations.py new file mode 100644 index 0000000000..700b77421a --- /dev/null +++ b/docs/src/python/operations.py @@ -0,0 +1,21 @@ +def replace_where(): + # --8<-- [start:replace_where] + import pyarrow as pa + from deltalake import write_deltalake + + # Assuming there is already a table in this location with some records where `id = '1'` which we want to overwrite + table_path = "/tmp/my_table" + data = pa.table( + { + "id": pa.array(["1", "1"], pa.string()), + "value": pa.array([11, 12], pa.int64()), + } + ) + write_deltalake( + table_path, + data, + mode="overwrite", + predicate="id = '1'", + engine="rust", + ) + # --8<-- [end:replace_where] diff --git a/docs/src/rust/check_constraints.rs b/docs/src/rust/check_constraints.rs new file mode 100644 index 0000000000..fbc2cf18d6 --- /dev/null +++ b/docs/src/rust/check_constraints.rs @@ -0,0 +1,23 @@ +use std::sync::Arc; + +#[tokio::main] +async fn main() -> Result<(), Box> { + + // --8<-- [start:add_constraint] + let table = deltalake::open_table("../rust/tests/data/simple_table").await?; + let ops = DeltaOps(table); + ops.with_constraint("id_gt_0", "id > 0").await?; + // --8<-- [end:add_constraint] + + // --8<-- [start:add_data] + let table = deltalake::open_table("../rust/tests/data/simple_table").await?; + let schema = table.get_state().arrow_schema()?; + let invalid_values: Vec> = vec![ + Arc::new(Int32Array::from(vec![-10])) + ]; + let batch = RecordBatch::try_new(schema, invalid_values)?; + table.write(vec![batch]).await?; + // --8<-- [end:add_data] + + Ok(()) +} \ No newline at end of file diff --git a/docs/src/rust/operations.rs b/docs/src/rust/operations.rs new file mode 100644 index 0000000000..55ab40604f --- /dev/null +++ b/docs/src/rust/operations.rs @@ -0,0 +1,32 @@ +#[tokio::main] +async fn main() -> Result<(), Box> { + // --8<-- [start:replace_where] + // Assuming there is already a table in this location with some records where `id = '1'` which we want to overwrite + use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + use arrow_array::RecordBatch; + import deltalake::protocol::SaveMode; + + let schema = ArrowSchema::new(vec![ + Field::new("id", DataType::Utf8, true), + Field::new("value", DataType::Int32, true), + ]); + + let data = RecordBatch::try_new( + &schema, + vec![ + Arc::new(arrow::array::StringArray::from(vec!["1", "1"])), + Arc::new(arrow::array::Int32Array::from(vec![11, 12])), + ], + ) + .unwrap(); + + let table = deltalake::open_table("/tmp/my_table").await.unwrap(); + let table = DeltaOps(table) + .write(vec![data]) + .with_save_mode(SaveMode::Overwrite) + .with_replace_where(col("id").eq(lit("1"))) + .await; + // --8<-- [end:replace_where] + + Ok(()) +} \ No newline at end of file diff --git a/docs/usage/constraints.md b/docs/usage/constraints.md new file mode 100644 index 0000000000..3e8bc8122c --- /dev/null +++ b/docs/usage/constraints.md @@ -0,0 +1,17 @@ +# Adding a Constraint to a table + +Check constraints are a way to enforce that only data that meets the constraint is allowed to be added to the table. + +## Add the Constraint + +{{ code_example('check_constraints', 'add_constraint', ['DeltaTable']) }} + +After you have added the constraint to the table attempting to append data to the table that violates the constraint +will instead throw an error. + +## Verify the constraint by trying to add some data + +{{ code_example('check_constraints', 'add_data', []) }} + +Note: ensure you use the `engine='rust'` parameter when writing to the table as this feature is not supported in the +default pyarrow writer. \ No newline at end of file diff --git a/docs/usage/create-delta-lake-table.md b/docs/usage/create-delta-lake-table.md index 3a2f023a47..2d76ac4cf7 100644 --- a/docs/usage/create-delta-lake-table.md +++ b/docs/usage/create-delta-lake-table.md @@ -4,13 +4,25 @@ This section explains how to create a Delta Lake table. You can easily write a DataFrame to a Delta table. -```python -from deltalake import write_deltalake -import pandas as pd -df = pd.DataFrame({"num": [1, 2, 3], "letter": ["a", "b", "c"]}) -write_deltalake("tmp/some-table", df) -``` +=== "pandas" + + ```python + from deltalake import write_deltalake + import pandas as pd + + df = pd.DataFrame({"num": [1, 2, 3], "letter": ["a", "b", "c"]}) + write_deltalake("tmp/some-table", df) + ``` + +=== "Polars" + + ```python + import polars as pl + + df = pl.DataFrame({"num": [1, 2, 3], "letter": ["a", "b", "c"]}) + df.write_delta("tmp/some-table") + ``` Here are the contents of the Delta table in storage: diff --git a/docs/usage/examining-table.md b/docs/usage/examining-table.md index 9c99450078..dbe5e2c585 100644 --- a/docs/usage/examining-table.md +++ b/docs/usage/examining-table.md @@ -75,9 +75,13 @@ default, unless otherwise specified by the table configuration To view the available history, use `DeltaTable.history`: ``` python ->>> from deltalake import DeltaTable ->>> dt = DeltaTable("../rust/tests/data/simple_table") ->>> dt.history() +from deltalake import DeltaTable + +dt = DeltaTable("../rust/tests/data/simple_table") +dt.history() +``` + +``` [{'timestamp': 1587968626537, 'operation': 'DELETE', 'operationParameters': {'predicate': '["((`id` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT))"]'}, 'readVersion': 3, 'isBlindAppend': False}, {'timestamp': 1587968614187, 'operation': 'UPDATE', 'operationParameters': {'predicate': '((id#697L % cast(2 as bigint)) = cast(0 as bigint))'}, 'readVersion': 2, 'isBlindAppend': False}, {'timestamp': 1587968604143, 'operation': 'WRITE', 'operationParameters': {'mode': 'Overwrite', 'partitionBy': '[]'}, 'readVersion': 1, 'isBlindAppend': False}, diff --git a/docs/usage/index.md b/docs/usage/index.md deleted file mode 100644 index cfce5d0390..0000000000 --- a/docs/usage/index.md +++ /dev/null @@ -1,8 +0,0 @@ -# Usage - -A [DeltaTable][deltalake.table.DeltaTable] represents the state of a -delta table at a particular version. This includes which files are -currently part of the table, the schema of the table, and other metadata -such as creation time. - -{{ code_example('delta_table', 'get_table_info', ['DeltaTable'])}} \ No newline at end of file diff --git a/docs/usage/loading-table.md b/docs/usage/loading-table.md index 1af7df95e5..e8d817c6e5 100644 --- a/docs/usage/loading-table.md +++ b/docs/usage/loading-table.md @@ -1,10 +1,11 @@ # Loading a Delta Table -To load the current version, use the constructor: +A [DeltaTable][deltalake.table.DeltaTable] represents the state of a +delta table at a particular version. This includes which files are +currently part of the table, the schema of the table, and other metadata +such as creation time. -``` python ->>> dt = DeltaTable("../rust/tests/data/delta-0.2.0") -``` +{{ code_example('delta_table', 'get_table_info', ['DeltaTable'])}} Depending on your storage backend, you could use the `storage_options` parameter to provide some configuration. Configuration is defined for diff --git a/docs/usage/overview.md b/docs/usage/overview.md new file mode 100644 index 0000000000..6fb269bda2 --- /dev/null +++ b/docs/usage/overview.md @@ -0,0 +1,7 @@ +# Usage + +This guide teaches you how to use Delta Lake. You will learn how to create Delta tables, run queries, perform DML operations, and optimize your tables. + +It's easy to use Delta Lake with pandas, Polars, Rust, or any other PyArrow-like DataFrame library. + +See the [Spark Delta Lake documentation](https://docs.delta.io/latest/index.html) if you're using Delta Lake with Spark. diff --git a/docs/usage/writing-delta-tables.md b/docs/usage/writing/index.md similarity index 64% rename from docs/usage/writing-delta-tables.md rename to docs/usage/writing/index.md index fe92572a81..9e9e1bcbec 100644 --- a/docs/usage/writing-delta-tables.md +++ b/docs/usage/writing/index.md @@ -23,7 +23,9 @@ of Spark's `pyspark.sql.DataFrameWriter.saveAsTable` DataFrame method. To overwr `write_deltalake` will raise `ValueError` if the schema of the data passed to it differs from the existing table's schema. If you wish to -alter the schema as part of an overwrite pass in `overwrite_schema=True`. +alter the schema as part of an overwrite pass in `schema_mode="overwrite"` or `schema_mode="merge"`. +`schema_mode="overwrite"` will completely overwrite the schema, even if columns are dropped; merge will append the new columns +and fill missing columns with `null`. `schema_mode="merge"` is also supported on append operations. ## Overwriting a partition @@ -50,4 +52,21 @@ that partition or else the method will raise an error. ``` This method could also be used to insert a new partition if one doesn't -already exist, making this operation idempotent. \ No newline at end of file +already exist, making this operation idempotent. + +## Overwriting part of the table data using a predicate + +!!! note + + This predicate is often called a `replaceWhere` predicate + +When you don’t specify the `predicate`, the overwrite save mode will replace the entire table. +Instead of replacing the entire table (which is costly!), you may want to overwrite only the specific parts of the table that should be changed. +In this case, you can use a `predicate` to overwrite only the relevant records or partitions. + +!!! note + + Data written must conform to the same predicate, i.e. not contain any records that don't match the `predicate` condition, + otherwise the operation will fail + +{{ code_example('operations', 'replace_where', ['replaceWhere'])}} \ No newline at end of file diff --git a/docs/usage/writing/writing-to-s3-with-locking-provider.md b/docs/usage/writing/writing-to-s3-with-locking-provider.md new file mode 100644 index 0000000000..99dcd4f23b --- /dev/null +++ b/docs/usage/writing/writing-to-s3-with-locking-provider.md @@ -0,0 +1,67 @@ +# Writing to S3 with a locking provider + +A locking mechanism is needed to prevent unsafe concurrent writes to a +delta lake directory when writing to S3. + +### DynamoDB +DynamoDB is the only available locking provider at the moment in delta-rs. To enable DynamoDB as the locking provider, you need to set the ``AWS_S3_LOCKING_PROVIDER`` to 'dynamodb' as a ``storage_options`` or as an environment variable. + +Additionally, you must create a DynamoDB table with the name ``delta_log`` +so that it can be automatically recognized by delta-rs. Alternatively, you can +use a table name of your choice, but you must set the ``DELTA_DYNAMO_TABLE_NAME`` +variable to match your chosen table name. The required schema for the DynamoDB +table is as follows: + +```json +"Table": { + "AttributeDefinitions": [ + { + "AttributeName": "fileName", + "AttributeType": "S" + }, + { + "AttributeName": "tablePath", + "AttributeType": "S" + } + ], + "TableName": "delta_log", + "KeySchema": [ + { + "AttributeName": "tablePath", + "KeyType": "HASH" + }, + { + "AttributeName": "fileName", + "KeyType": "RANGE" + } + ], +} +``` + +Here is an example writing to s3 using this mechanism: + +```python +from deltalake import write_deltalake +df = pd.DataFrame({'x': [1, 2, 3]}) +storage_options = {'AWS_S3_LOCKING_PROVIDER': 'dynamodb', 'DELTA_DYNAMO_TABLE_NAME': 'custom_table_name'} +write_deltalake('s3a://path/to/table', df, 'storage_options'= storage_options) +``` + +This locking mechanism is compatible with the one used by Apache Spark. The `tablePath` property, denoting the root url of the delta table itself, is part of the primary key, and all writers intending to write to the same table must match this property precisely. In Spark, S3 URLs are prefixed with `s3a://`, and a table in delta-rs must be configured accordingly. + +The following code allows creating the necessary table from the AWS cli: + +```sh +aws dynamodb create-table \ +--table-name delta_log \ +--attribute-definitions AttributeName=tablePath,AttributeType=S AttributeName=fileName,AttributeType=S \ +--key-schema AttributeName=tablePath,KeyType=HASH AttributeName=fileName,KeyType=RANGE \ +--provisioned-throughput ReadCapacityUnits=5,WriteCapacityUnits=5 +``` + +You can find additional information in the [delta-rs-documentation](https://docs.delta.io/latest/delta-storage.html#multi-cluster-setup), which also includes recommendations on configuring a time-to-live (TTL) for the table to avoid growing the table indefinitely. + + +### Enable unsafe writes in S3 (opt-in) +If for some reason you don't want to use dynamodb as your locking mechanism you can +choose to set the `AWS_S3_ALLOW_UNSAFE_RENAME` variable to ``true`` in order to enable S3 unsafe writes. \ No newline at end of file diff --git a/docs/why-use-delta-lake.md b/docs/why-use-delta-lake.md new file mode 100644 index 0000000000..4ecd468fa1 --- /dev/null +++ b/docs/why-use-delta-lake.md @@ -0,0 +1,94 @@ +# Why use Delta Lake + +This page explains why Delta Lake is a better storage format for most tabular data analyses than data lake alternatives. + +Delta Lake provides developer-friendly features, reliable transactions, and fast performance compared with alternatives like Parquet or CSV. + +## Fast performance + +Delta tables store data in Parquet files and persist file-level metadata in the transaction log. + +This offers two main performance advantages: + +* File skipping based on metadata that’s quickly accessible +* Easy identification of all file paths for the table, compared to file listing operations that can be slow, especially on cloud object stores + +Delta Lake stores min/max values for each column of each file in the table. Certain queries can skip entire files based on the metadata. File skipping can be a massive performance optimization. + +Delta Lake also makes it easy to rearrange data in the table, so more file skipping is possible. For example, the table can be partitioned or Z Ordered, so that similar data is colocated in the same files and data skipping is optimal for your query patterns. + +For data lakes, you need to run file listing operations to get the file paths before you can actually read the data. Listing all the files in a data lake can take a long time, especially if there are a lot of files and they are stored in Hive-style partitions. + +Delta Lake stores all the file paths in the transaction log. So you can quickly get the file paths directly from the log and then run your query. Delta Lake also stores the file-level metadata in the transaction log which is quicker than opening all the files in the data lake and grabbing the metadata from the file footer. + +## Developer friendly features + +Many basic data operations are hard in data lakes but quite easy with Delta Lake. The only data operation that’s easy with in data lake is appending data. Delta Lake makes all data operations easy including the following: + +* Appends +* Upserts +* Deletes +* Replace where + +Even deleting a few rows of data from a data lake is hard. It’s even harder if you want to run the operation in a performant manner. + +Delta Lake makes it easy to run common data operations and executes them performantly under the hood. + +Delta Lake also executes write operations as transactions, which makes data operations safer and prevents downtime. Write operations will cause data lakes to be in an unstable state while the computations is running. For example, if you read a data lake while a delete operation is running, then you may get the wrong data. + +Let’s explore the benefits of reliable transactions in more detail. + +## Reliable transactions + +Delta Lake supports transactions which means that write operations have the following characteristics: + +* They either finish completely or don’t run at all +* They are executed in a serial manner and don’t conflict with other transactions +* They don’t corrupt a table or violate table constraints + +Data lakes don’t support transactions, so the write operations can cause the following errors: + +* There is no schema enforcement, so you can append data to a data lake with a mismatching schema +* Reading the data lake often yields incorrect results while write transactions are performed +* Data lakes can be corrupted for invalid write operations or computations that error-out +* Concurrent transactions that conflict can cause data loss + +Production data systems should rely on storage systems like Delta Lake that support transactions. + +## Interoperability + +Delta Lake tables are interoperable and can be read/written by multiple different query engines. + +For example, you can create a Delta table with Spark, append to it with pandas, and then read it with Polars. + +![](delta-interop.png) + +Delta tables are powerful because they are interoperable with various query engines and computation runtimes. + +Suppose you have a Delta table that’s updated with an AWS Lambda function every 5 minutes. There is only a small amount of data collected every 5 minutes, so a lightweight runtime like AWS Lambda is sufficient. + +Further suppose that the overall table is quite large. So when you want to perform DML operations or query the whole table, your team uses a Spark cluster. + +Delta Lake is flexible to allow these types of operations from multiple readers and writers. This provides teams with the flexibility to choose the right tool for the job. + +## Support for many languages + +Delta tables can be queried with a variety of different languages. This project provides APIs for Rust and Python users and does not depend on Java or Scala. This project is a great alternative for pandas, Polars, DuckDB, or DataFusion. + +Delta Lake supports many languages and even more language support is coming soon! + +## Support on multiple clouds + +Delta Lake supports multiple clouds including GCP, AWS, and Azure. + +You can also use Delta Lake on your local machine or in an on-prem environment. + +Delta Lake is quite portable. + +## Conclusion + +Delta Lake is a mature table format that offers users tons of advantages over a data lake with virtually no downsides. + +Once you start using Delta Lake, you will never want to go back to data lakes that expose you to a variety of dangerous bugs, poor performance, and reliability issues. + +The Delta Lake community is also welcome and open. We gladly accept new contributors and help users with their questions. diff --git a/logo.png b/logo.png deleted file mode 100644 index 240e63ad6e..0000000000 Binary files a/logo.png and /dev/null differ diff --git a/mkdocs.yml b/mkdocs.yml index 4e713d73ec..f751755009 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -45,17 +45,24 @@ theme: - content.code.annotation - content.code.copy nav: - - Home: index.md + - Home: + - Home: index.md + - Why Use Delta Lake: why-use-delta-lake.md + - Delta Lake for big and small data: delta-lake-big-data-small-data.md + - Best practices: delta-lake-best-practices.md - Usage: - Installation: usage/installation.md - - Overview: usage/index.md + - Overview: usage/overview.md - Creating a table: usage/create-delta-lake-table.md - Loading a table: usage/loading-table.md - Append/overwrite tables: usage/appending-overwriting-delta-lake-table.md + - Adding a constraint: usage/constraints.md - Examining a table: usage/examining-table.md - Querying a table: usage/querying-delta-tables.md - Managing a table: usage/managing-tables.md - - Writing a table: usage/writing-delta-tables.md + - Writing a table: + - usage/writing/index.md + - usage/writing/writing-to-s3-with-locking-provider.md - Deleting rows from a table: usage/deleting-rows-from-delta-lake-table.md - Optimize: - Small file compaction: usage/optimize/small-file-compaction-with-optimize.md @@ -67,16 +74,22 @@ nav: - api/delta_table/metadata.md - api/delta_table/delta_table_merger.md - api/delta_table/delta_table_optimizer.md + - api/delta_table/delta_table_alterer.md - api/schema.md - api/storage.md - api/catalog.md - api/exceptions.md - Integrations: - Arrow: integrations/delta-lake-arrow.md + - Dagster: integrations/delta-lake-dagster.md + - Dask: integrations/delta-lake-dask.md + - DataFusion: integrations/delta-lake-datafusion.md - pandas: integrations/delta-lake-pandas.md - Polars: integrations/delta-lake-polars.md - How Delta Lake works: - Architecture: how-delta-lake-works/architecture-of-delta-table.md + - Transactions: how-delta-lake-works/delta-lake-acid-transactions.md + - File skipping: how-delta-lake-works/delta-lake-file-skipping.md not_in_nav: | /_build/ diff --git a/python/CONTRIBUTING.md b/python/CONTRIBUTING.md index 66d8ed623c..2bf638c51f 100644 --- a/python/CONTRIBUTING.md +++ b/python/CONTRIBUTING.md @@ -7,12 +7,12 @@ Most of the workflow is based on the `Makefile` and the `maturin` CLI tool. #### Setup your local environment with virtualenv ```bash -$ make setup-venv +make setup-venv ``` #### Activate it ```bash -$ source ./venv/bin/activate +source ./venv/bin/activate ``` #### Ready to develop with maturin @@ -21,13 +21,13 @@ $ source ./venv/bin/activate Install delta-rs in the current virtualenv ```bash -$ make develop +make develop ``` Then, list all the available tasks ```bash -$ make help +make help ``` Format: diff --git a/python/Cargo.toml b/python/Cargo.toml index a9936a483c..486f37d695 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-python" -version = "0.14.0" +version = "0.16.0" authors = ["Qingping Hou ", "Will Jones "] homepage = "https://github.com/delta-io/delta-rs" license = "Apache-2.0" @@ -45,9 +45,9 @@ features = ["extension-module", "abi3", "abi3-py38"] [dependencies.deltalake] path = "../crates/deltalake" version = "0" -features = ["azure", "gcs", "python", "datafusion", "unity-experimental"] +features = ["azure", "gcs", "python", "datafusion", "unity-experimental", "mount"] [features] default = ["rustls"] -native-tls = ["deltalake/s3-native-tls", "deltalake/glue-native-tls"] +native-tls = ["deltalake/s3-native-tls", "deltalake/glue"] rustls = ["deltalake/s3", "deltalake/glue"] diff --git a/python/README.md b/python/README.md index 6911598c52..d75b1520e8 100644 --- a/python/README.md +++ b/python/README.md @@ -1,8 +1,8 @@ # Deltalake-python [![PyPI](https://img.shields.io/pypi/v/deltalake.svg?style=flat-square)](https://pypi.org/project/deltalake/) -[![userdoc](https://img.shields.io/badge/docs-user-blue)](https://delta-io.github.io/delta-rs/python/) -[![apidoc](https://img.shields.io/badge/docs-api-blue)](https://delta-io.github.io/delta-rs/python/api_reference.html) +[![userdoc](https://img.shields.io/badge/docs-user-blue)](https://delta-io.github.io/delta-rs/) +[![apidoc](https://img.shields.io/badge/docs-api-blue)](https://delta-io.github.io/delta-rs/api/delta_table/) Native [Delta Lake](https://delta.io/) Python binding based on [delta-rs](https://github.com/delta-io/delta-rs) with @@ -22,7 +22,7 @@ dt.files() 'part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet'] ``` -See the [user guide](https://delta-io.github.io/delta-rs/python/usage.html) for more examples. +See the [user guide](https://delta-io.github.io/delta-rs/usage/installation/) for more examples. ## Installation diff --git a/python/deltalake/__init__.py b/python/deltalake/__init__.py index b10a708309..99089ae922 100644 --- a/python/deltalake/__init__.py +++ b/python/deltalake/__init__.py @@ -6,5 +6,6 @@ from .schema import Schema as Schema from .table import DeltaTable as DeltaTable from .table import Metadata as Metadata +from .table import WriterProperties as WriterProperties from .writer import convert_to_deltalake as convert_to_deltalake from .writer import write_deltalake as write_deltalake diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 228488d91a..b16d468571 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -44,7 +44,7 @@ class RawDeltaTable: def version(self) -> int: ... def get_latest_version(self) -> int: ... def metadata(self) -> RawDeltaTableMetaData: ... - def protocol_versions(self) -> List[int]: ... + def protocol_versions(self) -> List[Any]: ... def load_version(self, version: int) -> None: ... def load_with_datetime(self, ds: str) -> None: ... def files_by_partitions( @@ -57,6 +57,7 @@ class RawDeltaTable: dry_run: bool, retention_hours: Optional[int], enforce_retention_duration: bool, + custom_metadata: Optional[Dict[str, str]], ) -> List[str]: ... def compact_optimize( self, @@ -64,6 +65,8 @@ class RawDeltaTable: target_size: Optional[int], max_concurrent_tasks: Optional[int], min_commit_interval: Optional[int], + writer_properties: Optional[Dict[str, Optional[str]]], + custom_metadata: Optional[Dict[str, str]], ) -> str: ... def z_order_optimize( self, @@ -73,12 +76,26 @@ class RawDeltaTable: max_concurrent_tasks: Optional[int], max_spill_size: Optional[int], min_commit_interval: Optional[int], + writer_properties: Optional[Dict[str, Optional[str]]], + custom_metadata: Optional[Dict[str, str]], ) -> str: ... + def add_constraints( + self, + constraints: Dict[str, str], + custom_metadata: Optional[Dict[str, str]], + ) -> None: ... + def drop_constraints( + self, + name: str, + raise_if_not_exists: bool, + custom_metadata: Optional[Dict[str, str]], + ) -> None: ... def restore( self, target: Optional[Any], ignore_missing_files: bool, protocol_downgrade_allowed: bool, + custom_metadata: Optional[Dict[str, str]], ) -> str: ... def history(self, limit: Optional[int]) -> List[str]: ... def update_incremental(self) -> None: ... @@ -87,14 +104,22 @@ class RawDeltaTable: ) -> List[Any]: ... def create_checkpoint(self) -> None: ... def get_add_actions(self, flatten: bool) -> pyarrow.RecordBatch: ... - def delete(self, predicate: Optional[str]) -> str: ... - def repair(self, dry_run: bool) -> str: ... + def delete( + self, + predicate: Optional[str], + writer_properties: Optional[Dict[str, Optional[str]]], + custom_metadata: Optional[Dict[str, str]], + ) -> str: ... + def repair( + self, dry_run: bool, custom_metadata: Optional[Dict[str, str]] + ) -> str: ... def update( self, updates: Dict[str, str], predicate: Optional[str], - writer_properties: Optional[Dict[str, int]], - safe_cast: bool = False, + writer_properties: Optional[Dict[str, Optional[str]]], + safe_cast: bool, + custom_metadata: Optional[Dict[str, str]], ) -> str: ... def merge_execute( self, @@ -102,7 +127,8 @@ class RawDeltaTable: predicate: str, source_alias: Optional[str], target_alias: Optional[str], - writer_properties: Optional[Dict[str, int | None]], + writer_properties: Optional[Dict[str, Optional[str]]], + custom_metadata: Optional[Dict[str, str]], safe_cast: bool, matched_update_updates: Optional[List[Dict[str, str]]], matched_update_predicate: Optional[List[Optional[str]]], @@ -125,6 +151,7 @@ class RawDeltaTable: partition_by: List[str], schema: pyarrow.Schema, partitions_filters: Optional[FilterType], + custom_metadata: Optional[Dict[str, str]], ) -> None: ... def cleanup_metadata(self) -> None: ... @@ -139,6 +166,7 @@ def write_new_deltalake( description: Optional[str], configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], + custom_metadata: Optional[Dict[str, str]], ) -> None: ... def write_to_deltalake( table_uri: str, @@ -146,12 +174,14 @@ def write_to_deltalake( partition_by: Optional[List[str]], mode: str, max_rows_per_group: int, - overwrite_schema: bool, + schema_mode: Optional[str], predicate: Optional[str], name: Optional[str], description: Optional[str], configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], + writer_properties: Optional[Dict[str, Optional[str]]], + custom_metadata: Optional[Dict[str, str]], ) -> None: ... def convert_to_deltalake( uri: str, @@ -172,6 +202,7 @@ def create_deltalake( description: Optional[str], configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], + custom_metadata: Optional[Dict[str, str]], ) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... @@ -201,6 +232,7 @@ class PrimitiveType: * "binary", * "date", * "timestamp", + * "timestampNtz", * "decimal(, )" Args: @@ -764,6 +796,11 @@ class DeltaProtocolError(DeltaError): pass +class SchemaMismatchError(DeltaError): + """Raised when a schema mismatch is detected.""" + + pass + FilterLiteralType = Tuple[str, str, Any] FilterConjunctionType = List[FilterLiteralType] FilterDNFType = List[FilterConjunctionType] diff --git a/python/deltalake/exceptions.py b/python/deltalake/exceptions.py index bacd0af9f8..a2e5b1ba1e 100644 --- a/python/deltalake/exceptions.py +++ b/python/deltalake/exceptions.py @@ -1,4 +1,5 @@ from ._internal import CommitFailedError as CommitFailedError from ._internal import DeltaError as DeltaError from ._internal import DeltaProtocolError as DeltaProtocolError +from ._internal import SchemaMismatchError as SchemaMismatchError from ._internal import TableNotFoundError as TableNotFoundError diff --git a/python/deltalake/schema.py b/python/deltalake/schema.py index a22725fdc5..7124d51de3 100644 --- a/python/deltalake/schema.py +++ b/python/deltalake/schema.py @@ -46,14 +46,17 @@ def _convert_pa_schema_to_delta( def dtype_to_delta_dtype(dtype: pa.DataType) -> pa.DataType: # Handle nested types - if isinstance(dtype, (pa.LargeListType, pa.ListType)): + if isinstance(dtype, (pa.LargeListType, pa.ListType, pa.FixedSizeListType)): return list_to_delta_dtype(dtype) elif isinstance(dtype, pa.StructType): return struct_to_delta_dtype(dtype) elif isinstance(dtype, pa.TimestampType): - return pa.timestamp( - "us" - ) # TODO(ion): propagate also timezone information during writeonce we can properly read TZ in delta schema + if dtype.tz is None: + return pa.timestamp("us") + else: + return pa.timestamp("us", "UTC") + elif type(dtype) is pa.FixedSizeBinaryType: + return pa.binary() try: return dtype_map[dtype] except KeyError: diff --git a/python/deltalake/table.py b/python/deltalake/table.py index e7b7613599..064ee3a83c 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -3,6 +3,7 @@ import warnings from dataclasses import dataclass from datetime import datetime, timedelta +from enum import Enum from functools import reduce from pathlib import Path from typing import ( @@ -24,6 +25,7 @@ import pyarrow import pyarrow.dataset as ds import pyarrow.fs as pa_fs +import pyarrow_hotfix # noqa: F401; addresses CVE-2023-47248; # type: ignore from pyarrow.dataset import ( Expression, FileSystemDataset, @@ -35,8 +37,6 @@ if TYPE_CHECKING: import os - import pandas - from deltalake._internal import DeltaDataChecker as _DeltaDataChecker from deltalake._internal import RawDeltaTable from deltalake._internal import create_deltalake as _create_deltalake @@ -46,8 +46,155 @@ from deltalake.fs import DeltaStorageHandler from deltalake.schema import Schema as DeltaSchema -MAX_SUPPORTED_READER_VERSION = 1 -MAX_SUPPORTED_WRITER_VERSION = 2 +try: + import pandas as pd # noqa: F811 +except ModuleNotFoundError: + _has_pandas = False +else: + _has_pandas = True + +MAX_SUPPORTED_PYARROW_WRITER_VERSION = 7 +NOT_SUPPORTED_PYARROW_WRITER_VERSIONS = [3, 4, 5, 6] +SUPPORTED_WRITER_FEATURES = {"appendOnly", "invariants", "timestampNtz"} + +MAX_SUPPORTED_READER_VERSION = 3 +NOT_SUPPORTED_READER_VERSION = 2 +SUPPORTED_READER_FEATURES = {"timestampNtz"} + + +class Compression(Enum): + UNCOMPRESSED = "UNCOMPRESSED" + SNAPPY = "SNAPPY" + GZIP = "GZIP" + BROTLI = "BROTLI" + LZ4 = "LZ4" + ZSTD = "ZSTD" + LZ4_RAW = "LZ4_RAW" + + @classmethod + def from_str(cls, value: str) -> "Compression": + try: + return cls(value.upper()) + except ValueError: + raise ValueError( + f"{value} is not a valid Compression. Valid values are: {[item.value for item in Compression]}" + ) + + def get_level_range(self) -> Tuple[int, int]: + if self == Compression.GZIP: + MIN_LEVEL = 0 + MAX_LEVEL = 10 + elif self == Compression.BROTLI: + MIN_LEVEL = 0 + MAX_LEVEL = 11 + elif self == Compression.ZSTD: + MIN_LEVEL = 1 + MAX_LEVEL = 22 + else: + raise KeyError(f"{self.value} does not have a compression level.") + return MIN_LEVEL, MAX_LEVEL + + def get_default_level(self) -> int: + if self == Compression.GZIP: + DEFAULT = 6 + elif self == Compression.BROTLI: + DEFAULT = 1 + elif self == Compression.ZSTD: + DEFAULT = 1 + else: + raise KeyError(f"{self.value} does not have a compression level.") + return DEFAULT + + def check_valid_level(self, level: int) -> bool: + MIN_LEVEL, MAX_LEVEL = self.get_level_range() + if level < MIN_LEVEL or level > MAX_LEVEL: + raise ValueError( + f"Compression level for {self.value} should fall between {MIN_LEVEL}-{MAX_LEVEL}" + ) + else: + return True + + +@dataclass(init=True) +class WriterProperties: + """A Writer Properties instance for the Rust parquet writer.""" + + def __init__( + self, + data_page_size_limit: Optional[int] = None, + dictionary_page_size_limit: Optional[int] = None, + data_page_row_count_limit: Optional[int] = None, + write_batch_size: Optional[int] = None, + max_row_group_size: Optional[int] = None, + compression: Optional[ + Literal[ + "UNCOMPRESSED", + "SNAPPY", + "GZIP", + "BROTLI", + "LZ4", + "ZSTD", + "LZ4_RAW", + ] + ] = None, + compression_level: Optional[int] = None, + ): + """Create a Writer Properties instance for the Rust parquet writer: + + Args: + data_page_size_limit: Limit DataPage size to this in bytes. + dictionary_page_size_limit: Limit the size of each DataPage to store dicts to this amount in bytes. + data_page_row_count_limit: Limit the number of rows in each DataPage. + write_batch_size: Splits internally to smaller batch size. + max_row_group_size: Max number of rows in row group. + compression: compression type. + compression_level: If none and compression has a level, the default level will be used, only relevant for + GZIP: levels (1-9), + BROTLI: levels (1-11), + ZSTD: levels (1-22), + """ + self.data_page_size_limit = data_page_size_limit + self.dictionary_page_size_limit = dictionary_page_size_limit + self.data_page_row_count_limit = data_page_row_count_limit + self.write_batch_size = write_batch_size + self.max_row_group_size = max_row_group_size + self.compression = None + + if compression_level is not None and compression is None: + raise ValueError( + """Providing a compression level without the compression type is not possible, + please provide the compression as well.""" + ) + if isinstance(compression, str): + compression_enum = Compression.from_str(compression) + if compression_enum in [ + Compression.GZIP, + Compression.BROTLI, + Compression.ZSTD, + ]: + if compression_level is not None: + if compression_enum.check_valid_level(compression_level): + parquet_compression = ( + f"{compression_enum.value}({compression_level})" + ) + else: + parquet_compression = f"{compression_enum.value}({compression_enum.get_default_level()})" + else: + parquet_compression = compression_enum.value + self.compression = parquet_compression + + def __str__(self) -> str: + return ( + f"WriterProperties(data_page_size_limit: {self.data_page_size_limit}, dictionary_page_size_limit: {self.dictionary_page_size_limit}, " + f"data_page_row_count_limit: {self.data_page_row_count_limit}, write_batch_size: {self.write_batch_size}, " + f"max_row_group_size: {self.max_row_group_size}, compression: {self.compression})" + ) + + def _to_dict(self) -> Dict[str, Optional[str]]: + values = {} + for key, value in self.__dict__.items(): + values[key] = str(value) if isinstance(value, int) else value + return values @dataclass(init=False) @@ -100,6 +247,8 @@ def __str__(self) -> str: class ProtocolVersions(NamedTuple): min_reader_version: int min_writer_version: int + writer_features: Optional[List[str]] + reader_features: Optional[List[str]] FilterLiteralType = Tuple[str, str, Any] @@ -263,7 +412,6 @@ def __init__( without_files=without_files, log_buffer_size=log_buffer_size, ) - self._metadata = Metadata(self._table) @classmethod def from_data_catalog( @@ -311,6 +459,7 @@ def create( description: Optional[str] = None, configuration: Optional[Mapping[str, Optional[str]]] = None, storage_options: Optional[Dict[str, str]] = None, + custom_metadata: Optional[Dict[str, str]] = None, ) -> "DeltaTable": """`CREATE` or `CREATE_OR_REPLACE` a delta table given a table_uri. @@ -326,6 +475,7 @@ def create( description: User-provided description for this table. configuration: A map containing configuration options for the metadata action. storage_options: options passed to the object store crate. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: DeltaTable: created delta table @@ -363,6 +513,7 @@ def create( description, configuration, storage_options, + custom_metadata, ) return cls(table_uri=table_uri, storage_options=storage_options) @@ -452,13 +603,59 @@ def file_uris( file_uris.__doc__ = "" + def load_as_version(self, version: Union[int, str, datetime]) -> None: + """ + Load/time travel a DeltaTable to a specified version number, or a timestamp version of the table. If a + string is passed then the argument should be an RFC 3339 and ISO 8601 date and time string format. + + Args: + version: the identifier of the version of the DeltaTable to load + + Example: + **Use a version number** + ``` + dt = DeltaTable("test_table") + dt.load_as_version(1) + ``` + + **Use a datetime object** + ``` + dt.load_as_version(datetime(2023,1,1)) + ``` + + **Use a datetime in string format** + ``` + dt.load_as_version("2018-01-26T18:30:09Z") + dt.load_as_version("2018-12-19T16:39:57-08:00") + dt.load_as_version("2018-01-26T18:30:09.453+00:00") + ``` + """ + if isinstance(version, int): + self._table.load_version(version) + elif isinstance(version, datetime): + self._table.load_with_datetime(version.isoformat()) + elif isinstance(version, str): + self._table.load_with_datetime(version) + else: + raise TypeError( + "Invalid datatype provided for version, only int, str or datetime are accepted." + ) + def load_version(self, version: int) -> None: """ Load a DeltaTable with a specified version. + !!! warning "Deprecated" + Load_version and load_with_datetime have been combined into `DeltaTable.load_as_version`. + Args: version: the identifier of the version of the DeltaTable to load """ + warnings.warn( + "Call to deprecated method DeltaTable.load_version. Use DeltaTable.load_as_version() instead.", + category=DeprecationWarning, + stacklevel=2, + ) self._table.load_version(version) def load_with_datetime(self, datetime_string: str) -> None: @@ -466,6 +663,9 @@ def load_with_datetime(self, datetime_string: str) -> None: Time travel Delta table to the latest version that's created at or before provided `datetime_string` argument. The `datetime_string` argument should be an RFC 3339 and ISO 8601 date and time string. + !!! warning "Deprecated" + Load_version and load_with_datetime have been combined into `DeltaTable.load_as_version`. + Args: datetime_string: the identifier of the datetime point of the DeltaTable to load @@ -476,6 +676,11 @@ def load_with_datetime(self, datetime_string: str) -> None: "2018-01-26T18:30:09.453+00:00" ``` """ + warnings.warn( + "Call to deprecated method DeltaTable.load_with_datetime. Use DeltaTable.load_as_version() instead.", + category=DeprecationWarning, + stacklevel=2, + ) self._table.load_with_datetime(datetime_string) @property @@ -498,7 +703,7 @@ def metadata(self) -> Metadata: Returns: the current Metadata registered in the transaction log """ - return self._metadata + return Metadata(self._table) def protocol(self) -> ProtocolVersions: """ @@ -529,8 +734,7 @@ def _backwards_enumerate( yield n, elem n -= 1 - commits = list(reversed(self._table.history(limit))) - + commits = list(self._table.history(limit)) history = [] for version, commit_info_raw in _backwards_enumerate( commits, start_end=self._table.get_latest_version() @@ -545,6 +749,7 @@ def vacuum( retention_hours: Optional[int] = None, dry_run: bool = True, enforce_retention_duration: bool = True, + custom_metadata: Optional[Dict[str, str]] = None, ) -> List[str]: """ Run the Vacuum command on the Delta Table: list and delete files no longer referenced by the Delta table and are older than the retention threshold. @@ -553,7 +758,7 @@ def vacuum( retention_hours: the retention threshold in hours, if none then the value from `configuration.deletedFileRetentionDuration` is used or default of 1 week otherwise. dry_run: when activated, list only the files, delete otherwise enforce_retention_duration: when disabled, accepts retention hours smaller than the value from `configuration.deletedFileRetentionDuration`. - + custom_metadata: custom metadata that will be added to the transaction commit. Returns: the list of files no longer referenced by the Delta Table and are older than the retention threshold. """ @@ -565,6 +770,7 @@ def vacuum( dry_run, retention_hours, enforce_retention_duration, + custom_metadata, ) def update( @@ -574,8 +780,9 @@ def update( Dict[str, Union[int, float, str, datetime, bool, List[Any]]] ] = None, predicate: Optional[str] = None, - writer_properties: Optional[Dict[str, int]] = None, + writer_properties: Optional[WriterProperties] = None, error_on_type_mismatch: bool = True, + custom_metadata: Optional[Dict[str, str]] = None, ) -> Dict[str, Any]: """`UPDATE` records in the Delta Table that matches an optional predicate. Either updates or new_values needs to be passed for it to execute. @@ -584,11 +791,9 @@ def update( updates: a mapping of column name to update SQL expression. new_values: a mapping of column name to python datatype. predicate: a logical expression. - writer_properties: Pass writer properties to the Rust parquet writer, see options https://arrow.apache.org/rust/parquet/file/properties/struct.WriterProperties.html, - only the following fields are supported: `data_page_size_limit`, `dictionary_page_size_limit`, - `data_page_row_count_limit`, `write_batch_size`, `max_row_group_size`. + writer_properties: Pass writer properties to the Rust parquet writer. error_on_type_mismatch: specify if update will return error if data types are mismatching :default = True - + custom_metadata: custom metadata that will be added to the transaction commit. Returns: the metrics from update @@ -648,7 +853,6 @@ def update( updates[key] = value elif updates is not None and new_values is None: for key, value in updates.items(): - print(type(key), type(value)) if not isinstance(value, str) or not isinstance(key, str): raise TypeError( f"The values of the updates parameter must all be SQL strings. Got {updates}. Did you mean to use the new_values parameter?" @@ -665,8 +869,9 @@ def update( metrics = self._table.update( updates, predicate, - writer_properties, + writer_properties._to_dict() if writer_properties else None, safe_cast=not error_on_type_mismatch, + custom_metadata=custom_metadata, ) return json.loads(metrics) @@ -674,8 +879,24 @@ def update( def optimize( self, ) -> "TableOptimizer": + """Namespace for all table optimize related methods. + + Returns: + TableOptimizer: TableOptimizer Object + """ return TableOptimizer(self) + @property + def alter( + self, + ) -> "TableAlterer": + """Namespace for all table alter related methods. + + Returns: + TableAlterer: TableAlterer Object + """ + return TableAlterer(self) + def merge( self, source: Union[ @@ -683,12 +904,15 @@ def merge( pyarrow.RecordBatch, pyarrow.RecordBatchReader, ds.Dataset, - "pandas.DataFrame", + "pd.DataFrame", ], predicate: str, source_alias: Optional[str] = None, target_alias: Optional[str] = None, error_on_type_mismatch: bool = True, + writer_properties: Optional[WriterProperties] = None, + large_dtypes: bool = True, + custom_metadata: Optional[Dict[str, str]] = None, ) -> "TableMerger": """Pass the source data which you want to merge on the target delta table, providing a predicate in SQL query like format. You can also specify on what to do when the underlying data types do not @@ -700,6 +924,9 @@ def merge( source_alias: Alias for the source table target_alias: Alias for the target table error_on_type_mismatch: specify if merge will return error if data types are mismatching :default = True + writer_properties: Pass writer properties to the Rust parquet writer + large_dtypes: If True, the data schema is kept in large_dtypes. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: TableMerger: TableMerger Object @@ -715,16 +942,16 @@ def merge( ) if isinstance(source, pyarrow.RecordBatchReader): - source = convert_pyarrow_recordbatchreader(source, large_dtypes=True) + source = convert_pyarrow_recordbatchreader(source, large_dtypes) elif isinstance(source, pyarrow.RecordBatch): - source = convert_pyarrow_recordbatch(source, large_dtypes=True) + source = convert_pyarrow_recordbatch(source, large_dtypes) elif isinstance(source, pyarrow.Table): - source = convert_pyarrow_table(source, large_dtypes=True) + source = convert_pyarrow_table(source, large_dtypes) elif isinstance(source, ds.Dataset): - source = convert_pyarrow_dataset(source, large_dtypes=True) - elif isinstance(source, pandas.DataFrame): + source = convert_pyarrow_dataset(source, large_dtypes) + elif _has_pandas and isinstance(source, pd.DataFrame): source = convert_pyarrow_table( - pyarrow.Table.from_pandas(source), large_dtypes=True + pyarrow.Table.from_pandas(source), large_dtypes ) else: raise TypeError( @@ -746,6 +973,8 @@ def validate_batch(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: source_alias=source_alias, target_alias=target_alias, safe_cast=not error_on_type_mismatch, + writer_properties=writer_properties, + custom_metadata=custom_metadata, ) def restore( @@ -754,6 +983,7 @@ def restore( *, ignore_missing_files: bool = False, protocol_downgrade_allowed: bool = False, + custom_metadata: Optional[Dict[str, str]] = None, ) -> Dict[str, Any]: """ Run the Restore command on the Delta Table: restore table to a given version or datetime. @@ -762,6 +992,7 @@ def restore( target: the expected version will restore, which represented by int, date str or datetime. ignore_missing_files: whether the operation carry on when some data files missing. protocol_downgrade_allowed: whether the operation when protocol version upgraded. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: the metrics from restore. @@ -771,12 +1002,14 @@ def restore( target.isoformat(), ignore_missing_files=ignore_missing_files, protocol_downgrade_allowed=protocol_downgrade_allowed, + custom_metadata=custom_metadata, ) else: metrics = self._table.restore( target, ignore_missing_files=ignore_missing_files, protocol_downgrade_allowed=protocol_downgrade_allowed, + custom_metadata=custom_metadata, ) return json.loads(metrics) @@ -799,11 +1032,26 @@ def to_pyarrow_dataset( Returns: the PyArrow dataset in PyArrow """ - if self.protocol().min_reader_version > MAX_SUPPORTED_READER_VERSION: + table_protocol = self.protocol() + if ( + table_protocol.min_reader_version > MAX_SUPPORTED_READER_VERSION + or table_protocol.min_reader_version == NOT_SUPPORTED_READER_VERSION + ): raise DeltaProtocolError( - f"The table's minimum reader version is {self.protocol().min_reader_version} " - f"but deltalake only supports up to version {MAX_SUPPORTED_READER_VERSION}." + f"The table's minimum reader version is {table_protocol.min_reader_version} " + f"but deltalake only supports version 1 or {MAX_SUPPORTED_READER_VERSION} with these reader features: {SUPPORTED_READER_FEATURES}" + ) + if ( + table_protocol.min_reader_version >= 3 + and table_protocol.reader_features is not None + ): + missing_features = {*table_protocol.reader_features}.difference( + SUPPORTED_READER_FEATURES ) + if len(missing_features) > 0: + raise DeltaProtocolError( + f"The table has set these reader features: {missing_features} but these are not yet supported by the deltalake reader." + ) if not filesystem: file_sizes = self.get_add_actions().to_pydict() @@ -873,7 +1121,7 @@ def to_pandas( columns: Optional[List[str]] = None, filesystem: Optional[Union[str, pa_fs.FileSystem]] = None, filters: Optional[FilterType] = None, - ) -> "pandas.DataFrame": + ) -> "pd.DataFrame": """ Build a pandas dataframe using data from the DeltaTable. @@ -963,7 +1211,12 @@ def get_add_actions(self, flatten: bool = False) -> pyarrow.RecordBatch: """ return self._table.get_add_actions(flatten) - def delete(self, predicate: Optional[str] = None) -> Dict[str, Any]: + def delete( + self, + predicate: Optional[str] = None, + writer_properties: Optional[WriterProperties] = None, + custom_metadata: Optional[Dict[str, str]] = None, + ) -> Dict[str, Any]: """Delete records from a Delta Table that statisfy a predicate. When a predicate is not provided then all records are deleted from the Delta @@ -973,14 +1226,22 @@ def delete(self, predicate: Optional[str] = None) -> Dict[str, Any]: Args: predicate: a SQL where clause. If not passed, will delete all rows. + writer_properties: Pass writer properties to the Rust parquet writer. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: the metrics from delete. """ - metrics = self._table.delete(predicate) + metrics = self._table.delete( + predicate, + writer_properties._to_dict() if writer_properties else None, + custom_metadata, + ) return json.loads(metrics) - def repair(self, dry_run: bool = False) -> Dict[str, Any]: + def repair( + self, dry_run: bool = False, custom_metadata: Optional[Dict[str, str]] = None + ) -> Dict[str, Any]: """Repair the Delta Table by auditing active files that do not exist in the underlying filesystem and removes them. This can be useful when there are accidental deletions or corrupted files. @@ -990,6 +1251,7 @@ def repair(self, dry_run: bool = False) -> Dict[str, Any]: Args: dry_run: when activated, list only the files, otherwise add remove actions to transaction log. Defaults to False. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: The metrics from repair (FSCK) action. @@ -1004,7 +1266,7 @@ def repair(self, dry_run: bool = False) -> Dict[str, Any]: {'dry_run': False, 'files_removed': ['6-0d084325-6885-4847-b008-82c1cf30674c-0.parquet', 5-4fba1d3e-3e20-4de1-933d-a8e13ac59f53-0.parquet']} ``` """ - metrics = self._table.repair(dry_run) + metrics = self._table.repair(dry_run, custom_metadata) return json.loads(metrics) @@ -1019,6 +1281,8 @@ def __init__( source_alias: Optional[str] = None, target_alias: Optional[str] = None, safe_cast: bool = True, + writer_properties: Optional[WriterProperties] = None, + custom_metadata: Optional[Dict[str, str]] = None, ): self.table = table self.source = source @@ -1026,7 +1290,8 @@ def __init__( self.source_alias = source_alias self.target_alias = target_alias self.safe_cast = safe_cast - self.writer_properties: Optional[Dict[str, Optional[int]]] = None + self.writer_properties = writer_properties + self.custom_metadata = custom_metadata self.matched_update_updates: Optional[List[Dict[str, str]]] = None self.matched_update_predicate: Optional[List[Optional[str]]] = None self.matched_delete_predicate: Optional[List[str]] = None @@ -1034,9 +1299,9 @@ def __init__( self.not_matched_insert_updates: Optional[List[Dict[str, str]]] = None self.not_matched_insert_predicate: Optional[List[Optional[str]]] = None self.not_matched_by_source_update_updates: Optional[List[Dict[str, str]]] = None - self.not_matched_by_source_update_predicate: Optional[ - List[Optional[str]] - ] = None + self.not_matched_by_source_update_predicate: Optional[List[Optional[str]]] = ( + None + ) self.not_matched_by_source_delete_predicate: Optional[List[str]] = None self.not_matched_by_source_delete_all: Optional[bool] = None @@ -1048,7 +1313,10 @@ def with_writer_properties( write_batch_size: Optional[int] = None, max_row_group_size: Optional[int] = None, ) -> "TableMerger": - """Pass writer properties to the Rust parquet writer, see options https://arrow.apache.org/rust/parquet/file/properties/struct.WriterProperties.html: + """ + !!! warning "Deprecated" + Use `.merge(writer_properties = WriterProperties())` instead + Pass writer properties to the Rust parquet writer, see options https://arrow.apache.org/rust/parquet/file/properties/struct.WriterProperties.html: Args: data_page_size_limit: Limit DataPage size to this in bytes. @@ -1060,14 +1328,20 @@ def with_writer_properties( Returns: TableMerger: TableMerger Object """ - writer_properties = { + warnings.warn( + "Call to deprecated method TableMerger.with_writer_properties. Use DeltaTable.merge(writer_properties=WriterProperties()) instead.", + category=DeprecationWarning, + stacklevel=2, + ) + + writer_properties: Dict[str, Any] = { "data_page_size_limit": data_page_size_limit, "dictionary_page_size_limit": dictionary_page_size_limit, "data_page_row_count_limit": data_page_row_count_limit, "write_batch_size": write_batch_size, "max_row_group_size": max_row_group_size, } - self.writer_properties = writer_properties + self.writer_properties = WriterProperties(**writer_properties) return self def when_matched_update( @@ -1076,6 +1350,10 @@ def when_matched_update( """Update a matched table row based on the rules defined by ``updates``. If a ``predicate`` is specified, then it must evaluate to true for the row to be updated. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: updates: a mapping of column name to update SQL expression. predicate: SQL like predicate on when to update. @@ -1088,10 +1366,10 @@ def when_matched_update( from deltalake import DeltaTable, write_deltalake import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + data = pa.table({"x": [1, 2, 3], "1y": [4, 5, 6]}) write_deltalake("tmp", data) dt = DeltaTable("tmp") - new_data = pa.table({"x": [1], "y": [7]}) + new_data = pa.table({"x": [1], "1y": [7]}) ( dt.merge( @@ -1099,7 +1377,7 @@ def when_matched_update( predicate="target.x = source.x", source_alias="source", target_alias="target") - .when_matched_update(updates={"x": "source.x", "y": "source.y"}) + .when_matched_update(updates={"x": "source.x", "`1y`": "source.`1y`"}) .execute() ) {'num_source_rows': 1, 'num_target_rows_inserted': 0, 'num_target_rows_updated': 1, 'num_target_rows_deleted': 0, 'num_target_rows_copied': 2, 'num_output_rows': 3, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} @@ -1125,6 +1403,10 @@ def when_matched_update_all(self, predicate: Optional[str] = None) -> "TableMerg """Updating all source fields to target fields, source and target are required to have the same field names. If a ``predicate`` is specified, then it must evaluate to true for the row to be updated. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: predicate: SQL like predicate on when to update all columns. @@ -1164,7 +1446,7 @@ def when_matched_update_all(self, predicate: Optional[str] = None) -> "TableMerg trgt_alias = (self.target_alias + ".") if self.target_alias is not None else "" updates = { - f"{trgt_alias}{col.name}": f"{src_alias}{col.name}" + f"{trgt_alias}`{col.name}`": f"{src_alias}`{col.name}`" for col in self.source.schema } @@ -1183,6 +1465,10 @@ def when_matched_delete(self, predicate: Optional[str] = None) -> "TableMerger": """Delete a matched row from the table only if the given ``predicate`` (if specified) is true for the matched row. If not specified it deletes all matches. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: predicate (str | None, Optional): SQL like predicate on when to delete. @@ -1259,6 +1545,10 @@ def when_not_matched_insert( """Insert a new row to the target table based on the rules defined by ``updates``. If a ``predicate`` is specified, then it must evaluate to true for the new row to be inserted. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: updates (dict): a mapping of column name to insert SQL expression. predicate (str | None, Optional): SQL like predicate on when to insert. @@ -1318,6 +1608,10 @@ def when_not_matched_insert_all( required to have the same field names. If a ``predicate`` is specified, then it must evaluate to true for the new row to be inserted. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: predicate: SQL like predicate on when to insert. @@ -1357,7 +1651,7 @@ def when_not_matched_insert_all( src_alias = (self.source_alias + ".") if self.source_alias is not None else "" trgt_alias = (self.target_alias + ".") if self.target_alias is not None else "" updates = { - f"{trgt_alias}{col.name}": f"{src_alias}{col.name}" + f"{trgt_alias}`{col.name}`": f"{src_alias}`{col.name}`" for col in self.source.schema } if isinstance(self.not_matched_insert_updates, list) and isinstance( @@ -1377,6 +1671,10 @@ def when_not_matched_by_source_update( """Update a target row that has no matches in the source based on the rules defined by ``updates``. If a ``predicate`` is specified, then it must evaluate to true for the row to be updated. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: updates: a mapping of column name to update SQL expression. predicate: SQL like predicate on when to update. @@ -1431,6 +1729,10 @@ def when_not_matched_by_source_delete( """Delete a target row that has no matches in the source from the table only if the given ``predicate`` (if specified) is true for the target row. + Note: + Column names with special characters, such as numbers or spaces should be encapsulated + in backticks: "target.`123column`" or "target.`my column`" + Args: predicate: SQL like predicate on when to delete when not matched by source. @@ -1464,7 +1766,10 @@ def execute(self) -> Dict[str, Any]: source_alias=self.source_alias, target_alias=self.target_alias, safe_cast=self.safe_cast, - writer_properties=self.writer_properties, + writer_properties=self.writer_properties._to_dict() + if self.writer_properties + else None, + custom_metadata=self.custom_metadata, matched_update_updates=self.matched_update_updates, matched_update_predicate=self.matched_update_predicate, matched_delete_predicate=self.matched_delete_predicate, @@ -1480,6 +1785,81 @@ def execute(self) -> Dict[str, Any]: return json.loads(metrics) +class TableAlterer: + """API for various table alteration commands.""" + + def __init__(self, table: DeltaTable) -> None: + self.table = table + + def add_constraint( + self, + constraints: Dict[str, str], + custom_metadata: Optional[Dict[str, str]] = None, + ) -> None: + """ + Add constraints to the table. Limited to `single constraint` at once. + + Args: + constraints: mapping of constraint name to SQL-expression to evaluate on write + custom_metadata: custom metadata that will be added to the transaction commit. + Example: + ```python + from deltalake import DeltaTable + dt = DeltaTable("test_table_constraints") + dt.alter.add_constraint({ + "value_gt_5": "value > 5", + }) + ``` + + **Check configuration** + ``` + dt.metadata().configuration + {'delta.constraints.value_gt_5': 'value > 5'} + ``` + """ + if len(constraints.keys()) > 1: + raise ValueError( + """add_constraints is limited to a single constraint addition at once for now. + Please execute add_constraints multiple times with each time a different constraint.""" + ) + + self.table._table.add_constraints(constraints, custom_metadata) + + def drop_constraint( + self, + name: str, + raise_if_not_exists: bool = True, + custom_metadata: Optional[Dict[str, str]] = None, + ) -> None: + """ + Drop constraints from a table. Limited to `single constraint` at once. + + Args: + name: constraint name which to drop. + raise_if_not_exists: set if should raise if not exists. + custom_metadata: custom metadata that will be added to the transaction commit. + Example: + ```python + from deltalake import DeltaTable + dt = DeltaTable("test_table_constraints") + dt.metadata().configuration + {'delta.constraints.value_gt_5': 'value > 5'} + ``` + + **Drop the constraint** + ```python + dt.alter.drop_constraint(name = "value_gt_5") + ``` + + **Configuration after dropping** + ```python + dt.metadata().configuration + {} + ``` + """ + self.table._table.drop_constraints(name, raise_if_not_exists, custom_metadata) + + class TableOptimizer: """API for various table optimization commands.""" @@ -1511,6 +1891,8 @@ def compact( target_size: Optional[int] = None, max_concurrent_tasks: Optional[int] = None, min_commit_interval: Optional[Union[int, timedelta]] = None, + writer_properties: Optional[WriterProperties] = None, + custom_metadata: Optional[Dict[str, str]] = None, ) -> Dict[str, Any]: """ Compacts small files to reduce the total number of files in the table. @@ -1532,6 +1914,8 @@ def compact( min_commit_interval: minimum interval in seconds or as timedeltas before a new commit is created. Interval is useful for long running executions. Set to 0 or timedelta(0), if you want a commit per partition. + writer_properties: Pass writer properties to the Rust parquet writer. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: the metrics from optimize @@ -1556,7 +1940,12 @@ def compact( min_commit_interval = int(min_commit_interval.total_seconds()) metrics = self.table._table.compact_optimize( - partition_filters, target_size, max_concurrent_tasks, min_commit_interval + partition_filters, + target_size, + max_concurrent_tasks, + min_commit_interval, + writer_properties._to_dict() if writer_properties else None, + custom_metadata, ) self.table.update_incremental() return json.loads(metrics) @@ -1569,6 +1958,8 @@ def z_order( max_concurrent_tasks: Optional[int] = None, max_spill_size: int = 20 * 1024 * 1024 * 1024, min_commit_interval: Optional[Union[int, timedelta]] = None, + writer_properties: Optional[WriterProperties] = None, + custom_metadata: Optional[Dict[str, str]] = None, ) -> Dict[str, Any]: """ Reorders the data using a Z-order curve to improve data skipping. @@ -1577,7 +1968,7 @@ def z_order( Args: columns: the columns to use for Z-ordering. There must be at least one column. - partition_filters: the partition filters that will be used for getting the matched files + partition_filters: the partition filters that will be used for getting the matched files target_size: desired file size after bin-packing files, in bytes. If not provided, will attempt to read the table configuration value ``delta.targetFileSize``. If that value isn't set, will use default value of 256MB. @@ -1588,6 +1979,8 @@ def z_order( min_commit_interval: minimum interval in seconds or as timedeltas before a new commit is created. Interval is useful for long running executions. Set to 0 or timedelta(0), if you want a commit per partition. + writer_properties: Pass writer properties to the Rust parquet writer. + custom_metadata: custom metadata that will be added to the transaction commit. Returns: the metrics from optimize @@ -1618,6 +2011,8 @@ def z_order( max_concurrent_tasks, max_spill_size, min_commit_interval, + writer_properties._to_dict() if writer_properties else None, + custom_metadata, ) self.table.update_incremental() return json.loads(metrics) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index bb69fee457..feb1ff720e 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -31,6 +31,8 @@ else: from typing_extensions import Literal +import warnings + import pyarrow as pa import pyarrow.dataset as ds import pyarrow.fs as pa_fs @@ -48,7 +50,13 @@ convert_pyarrow_recordbatchreader, convert_pyarrow_table, ) -from .table import MAX_SUPPORTED_WRITER_VERSION, DeltaTable +from .table import ( + MAX_SUPPORTED_PYARROW_WRITER_VERSION, + NOT_SUPPORTED_PYARROW_WRITER_VERSIONS, + SUPPORTED_WRITER_FEATURES, + DeltaTable, + WriterProperties, +) try: import pandas as pd # noqa: F811 @@ -84,7 +92,6 @@ def write_deltalake( *, schema: Optional[Union[pa.Schema, DeltaSchema]] = ..., partition_by: Optional[Union[List[str], str]] = ..., - filesystem: Optional[pa_fs.FileSystem] = None, mode: Literal["error", "append", "overwrite", "ignore"] = ..., file_options: Optional[ds.ParquetFileWriteOptions] = ..., max_partitions: Optional[int] = ..., @@ -96,12 +103,13 @@ def write_deltalake( description: Optional[str] = ..., configuration: Optional[Mapping[str, Optional[str]]] = ..., overwrite_schema: bool = ..., + schema_mode: Optional[Literal["overwrite"]] = ..., storage_options: Optional[Dict[str, str]] = ..., partition_filters: Optional[List[Tuple[str, str, Any]]] = ..., large_dtypes: bool = ..., engine: Literal["pyarrow"] = ..., -) -> None: - ... + custom_metadata: Optional[Dict[str, str]] = ..., +) -> None: ... @overload @@ -118,18 +126,47 @@ def write_deltalake( *, schema: Optional[Union[pa.Schema, DeltaSchema]] = ..., partition_by: Optional[Union[List[str], str]] = ..., - mode: Literal["error", "append", "overwrite", "ignore"] = ..., - max_rows_per_group: int = ..., + mode: Literal["error", "append", "ignore"] = ..., + name: Optional[str] = ..., + description: Optional[str] = ..., + configuration: Optional[Mapping[str, Optional[str]]] = ..., + overwrite_schema: bool = ..., + schema_mode: Optional[Literal["merge", "overwrite"]] = ..., + storage_options: Optional[Dict[str, str]] = ..., + large_dtypes: bool = ..., + engine: Literal["rust"], + writer_properties: WriterProperties = ..., + custom_metadata: Optional[Dict[str, str]] = ..., +) -> None: ... + + +@overload +def write_deltalake( + table_or_uri: Union[str, Path, DeltaTable], + data: Union[ + "pd.DataFrame", + ds.Dataset, + pa.Table, + pa.RecordBatch, + Iterable[pa.RecordBatch], + RecordBatchReader, + ], + *, + schema: Optional[Union[pa.Schema, DeltaSchema]] = ..., + partition_by: Optional[Union[List[str], str]] = ..., + mode: Literal["overwrite"], name: Optional[str] = ..., description: Optional[str] = ..., configuration: Optional[Mapping[str, Optional[str]]] = ..., overwrite_schema: bool = ..., + schema_mode: Optional[Literal["merge", "overwrite"]] = ..., storage_options: Optional[Dict[str, str]] = ..., predicate: Optional[str] = ..., large_dtypes: bool = ..., engine: Literal["rust"], -) -> None: - ... + writer_properties: WriterProperties = ..., + custom_metadata: Optional[Dict[str, str]] = ..., +) -> None: ... def write_deltalake( @@ -145,7 +182,6 @@ def write_deltalake( *, schema: Optional[Union[pa.Schema, DeltaSchema]] = None, partition_by: Optional[Union[List[str], str]] = None, - filesystem: Optional[pa_fs.FileSystem] = None, mode: Literal["error", "append", "overwrite", "ignore"] = "error", file_options: Optional[ds.ParquetFileWriteOptions] = None, max_partitions: Optional[int] = None, @@ -157,39 +193,26 @@ def write_deltalake( description: Optional[str] = None, configuration: Optional[Mapping[str, Optional[str]]] = None, overwrite_schema: bool = False, + schema_mode: Optional[Literal["merge", "overwrite"]] = None, storage_options: Optional[Dict[str, str]] = None, partition_filters: Optional[List[Tuple[str, str, Any]]] = None, predicate: Optional[str] = None, large_dtypes: bool = False, engine: Literal["pyarrow", "rust"] = "pyarrow", + writer_properties: Optional[WriterProperties] = None, + custom_metadata: Optional[Dict[str, str]] = None, ) -> None: """Write to a Delta Lake table If the table does not already exist, it will be created. - This function only supports writer protocol version 2 currently. When - attempting to write to an existing table with a higher min_writer_version, - this function will throw DeltaProtocolError. - - Note that this function does NOT register this table in a data catalog. + The pyarrow writer supports protocol version 2 currently and won't be updated. + For higher protocol support use engine='rust', this will become the default + eventually. A locking mechanism is needed to prevent unsafe concurrent writes to a - delta lake directory when writing to S3. DynamoDB is the only available - locking provider at the moment in delta-rs. To enable DynamoDB as the - locking provider, you need to set the `AWS_S3_LOCKING_PROVIDER` to 'dynamodb' - as a storage_option or as an environment variable. - - Additionally, you must create a DynamoDB table with the name 'delta_rs_lock_table' - so that it can be automatically discovered by delta-rs. Alternatively, you can - use a table name of your choice, but you must set the `DYNAMO_LOCK_TABLE_NAME` - variable to match your chosen table name. The required schema for the DynamoDB - table is as follows: - - - Key Schema: AttributeName=key, KeyType=HASH - - Attribute Definitions: AttributeName=key, AttributeType=S - - Please note that this locking mechanism is not compatible with any other - locking mechanisms, including the one used by Spark. + delta lake directory when writing to S3. For more information on the setup, follow + this usage guide: https://delta-io.github.io/delta-rs/usage/writing/writing-to-s3-with-locking-provider/ Args: table_or_uri: URI of a table or a DeltaTable object. @@ -197,9 +220,6 @@ def write_deltalake( schema: Optional schema to write. partition_by: List of columns to partition the table by. Only required when creating a new table. - filesystem: Optional filesystem to pass to PyArrow. If not provided will - be inferred from uri. The file system has to be rooted in the table root. - Use the pyarrow.fs.SubTreeFileSystem, to adopt the root of pyarrow file systems. mode: How to handle existing data. Default is to error if table already exists. If 'append', will add new data. If 'overwrite', will replace table with new data. @@ -227,13 +247,16 @@ def write_deltalake( name: User-provided identifier for this table. description: User-provided description for this table. configuration: A map containing configuration options for the metadata action. - overwrite_schema: If True, allows updating the schema of the table. - storage_options: options passed to the native delta filesystem. Unused if 'filesystem' is defined. + overwrite_schema: Deprecated, use schema_mode instead. + schema_mode: If set to "overwrite", allows replacing the schema of the table. Set to "merge" to merge with existing schema. + storage_options: options passed to the native delta filesystem. predicate: When using `Overwrite` mode, replace data that matches a predicate. Only used in rust engine. partition_filters: the partition filters that will be used for partition overwrite. Only used in pyarrow engine. large_dtypes: If True, the data schema is kept in large_dtypes, has no effect on pandas dataframe input. engine: writer engine to write the delta table. `Rust` engine is still experimental but you may see up to 4x performance improvements over pyarrow. + writer_properties: Pass writer properties to the Rust parquet writer. + custom_metadata: Custom metadata to add to the commitInfo. """ table, table_uri = try_get_table_and_table_uri(table_or_uri, storage_options) if table is not None: @@ -243,12 +266,19 @@ def write_deltalake( table.update_incremental() __enforce_append_only(table=table, configuration=configuration, mode=mode) + if overwrite_schema: + schema_mode = "overwrite" + warnings.warn( + "overwrite_schema is deprecated, use schema_mode instead. ", + category=DeprecationWarning, + stacklevel=2, + ) if isinstance(partition_by, str): partition_by = [partition_by] if isinstance(schema, DeltaSchema): - schema = schema.to_pyarrow() + schema = schema.to_pyarrow(as_large_types=True) if isinstance(data, RecordBatchReader): data = convert_pyarrow_recordbatchreader(data, large_dtypes) @@ -289,42 +319,53 @@ def write_deltalake( partition_by=partition_by, mode=mode, max_rows_per_group=max_rows_per_group, - overwrite_schema=overwrite_schema, + schema_mode=schema_mode, predicate=predicate, name=name, description=description, configuration=configuration, storage_options=storage_options, + writer_properties=( + writer_properties._to_dict() if writer_properties else None + ), + custom_metadata=custom_metadata, ) if table: table.update_incremental() elif engine == "pyarrow": - # We need to write against the latest table version - if filesystem is not None: - raise NotImplementedError( - "Filesystem support is not yet implemented. #570" + if schema_mode == "merge": + raise ValueError( + "schema_mode 'merge' is not supported in pyarrow engine. Use engine=rust" ) - + # We need to write against the latest table version filesystem = pa_fs.PyFileSystem(DeltaStorageHandler(table_uri, storage_options)) + def sort_arrow_schema(schema: pa.schema) -> pa.schema: + sorted_cols = sorted(iter(schema), key=lambda x: (x.name, str(x.type))) + return pa.schema(sorted_cols) + if table: # already exists - if schema != table.schema().to_pyarrow( - as_large_types=large_dtypes - ) and not (mode == "overwrite" and overwrite_schema): + if sort_arrow_schema(schema) != sort_arrow_schema( + table.schema().to_pyarrow(as_large_types=large_dtypes) + ) and not (mode == "overwrite" and schema_mode == "overwrite"): raise ValueError( "Schema of data does not match table schema\n" f"Data schema:\n{schema}\nTable Schema:\n{table.schema().to_pyarrow(as_large_types=large_dtypes)}" ) if mode == "error": - raise AssertionError("DeltaTable already exists.") + raise FileExistsError( + "Delta table already exists, write mode set to error." + ) elif mode == "ignore": return current_version = table.version() - if partition_by: - assert partition_by == table.metadata().partition_columns + if partition_by and partition_by != table.metadata().partition_columns: + raise ValueError( + f"Partition columns should be {table.metadata().partition_columns} but is {partition_by}" + ) else: partition_by = table.metadata().partition_columns @@ -388,12 +429,29 @@ def visitor(written_file: Any) -> None: if table is not None: # We don't currently provide a way to set invariants # (and maybe never will), so only enforce if already exist. - if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: + table_protocol = table.protocol() + if ( + table_protocol.min_writer_version > MAX_SUPPORTED_PYARROW_WRITER_VERSION + or table_protocol.min_writer_version + in NOT_SUPPORTED_PYARROW_WRITER_VERSIONS + ): raise DeltaProtocolError( "This table's min_writer_version is " - f"{table.protocol().min_writer_version}, " - "but this method only supports version 2." + f"{table_protocol.min_writer_version}, " + f"""but this method only supports version 2 or 7 with at max these features {SUPPORTED_WRITER_FEATURES} enabled. + Try engine='rust' instead which supports more features and writer versions.""" + ) + if ( + table_protocol.min_writer_version >= 7 + and table_protocol.writer_features is not None + ): + missing_features = {*table_protocol.writer_features}.difference( + SUPPORTED_WRITER_FEATURES ) + if len(missing_features) > 0: + raise DeltaProtocolError( + f"The table has set these writer features: {missing_features} but these are not supported by the pyarrow writer. Please use engine='rust'." + ) invariants = table.schema().invariants checker = _DeltaDataChecker(invariants) @@ -403,12 +461,12 @@ def check_data_is_aligned_with_partition_filtering( ) -> None: if table is None: return - existed_partitions: FrozenSet[ - FrozenSet[Tuple[str, Optional[str]]] - ] = table._table.get_active_partitions() - allowed_partitions: FrozenSet[ - FrozenSet[Tuple[str, Optional[str]]] - ] = table._table.get_active_partitions(partition_filters) + existed_partitions: FrozenSet[FrozenSet[Tuple[str, Optional[str]]]] = ( + table._table.get_active_partitions() + ) + allowed_partitions: FrozenSet[FrozenSet[Tuple[str, Optional[str]]]] = ( + table._table.get_active_partitions(partition_filters) + ) partition_values = pa.RecordBatch.from_arrays( [ batch.column(column_name) @@ -487,6 +545,7 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: description, configuration, storage_options, + custom_metadata, ) else: table._table.create_write_transaction( @@ -495,6 +554,7 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: partition_by or [], schema, partition_filters, + custom_metadata, ) table.update_incremental() else: @@ -652,7 +712,8 @@ def get_file_stats_from_metadata( def iter_groups(metadata: Any) -> Iterator[Any]: for i in range(metadata.num_row_groups): - yield metadata.row_group(i) + if metadata.row_group(i).num_rows > 0: + yield metadata.row_group(i) for column_idx in range(metadata.num_columns): name = metadata.row_group(0).column(column_idx).path_in_schema diff --git a/python/docs/source/_ext/edit_on_github.py b/python/docs/source/_ext/edit_on_github.py index f7188f189a..241560877c 100644 --- a/python/docs/source/_ext/edit_on_github.py +++ b/python/docs/source/_ext/edit_on_github.py @@ -38,9 +38,9 @@ def html_page_context(app, pagename, templatename, context, doctree): context["display_github"] = True context["github_user"] = app.config.edit_on_github_project.split("/")[0] context["github_repo"] = app.config.edit_on_github_project.split("/")[1] - context[ - "github_version" - ] = f"{app.config.edit_on_github_branch}/{app.config.page_source_prefix}/" + context["github_version"] = ( + f"{app.config.edit_on_github_branch}/{app.config.page_source_prefix}/" + ) def setup(app): diff --git a/python/docs/source/index.rst b/python/docs/source/index.rst index 271d3a85a5..0ab6ad86a7 100644 --- a/python/docs/source/index.rst +++ b/python/docs/source/index.rst @@ -16,7 +16,7 @@ Pandas_, DuckDB_, and Polars_. It is not yet as feature-complete as the PySpark implementation of Delta Lake. If you encounter a bug, please let us know in our `GitHub repo`_. -.. _delta-spark: https://docs.delta.io/latest/api/python/index.html +.. _delta-spark: https://docs.delta.io/latest/api/python/spark/index.html .. _Delta Lake: https://delta.io/ .. _Apache Arrow: https://arrow.apache.org/ .. _Pandas: https://pandas.pydata.org/ diff --git a/python/docs/source/usage.rst b/python/docs/source/usage.rst index ed0556a176..baa26f275c 100644 --- a/python/docs/source/usage.rst +++ b/python/docs/source/usage.rst @@ -481,7 +481,7 @@ to append pass in ``mode='append'``: :py:meth:`write_deltalake` will raise :py:exc:`ValueError` if the schema of the data passed to it differs from the existing table's schema. If you wish to -alter the schema as part of an overwrite pass in ``overwrite_schema=True``. +alter the schema as part of an overwrite pass in ``schema_mode="overwrite"``. Writing to s3 ~~~~~~~~~~~~~ @@ -492,30 +492,39 @@ locking provider at the moment in delta-rs. To enable DynamoDB as the locking provider, you need to set the **AWS_S3_LOCKING_PROVIDER** to 'dynamodb' as a ``storage_options`` or as an environment variable. -Additionally, you must create a DynamoDB table with the name ``delta_rs_lock_table`` +Additionally, you must create a DynamoDB table with the name ``delta_log`` so that it can be automatically recognized by delta-rs. Alternatively, you can -use a table name of your choice, but you must set the **DYNAMO_LOCK_TABLE_NAME** +use a table name of your choice, but you must set the **DELTA_DYNAMO_TABLE_NAME** variable to match your chosen table name. The required schema for the DynamoDB table is as follows: .. code-block:: json - { + + "Table": { "AttributeDefinitions": [ { - "AttributeName": "key", + "AttributeName": "fileName", + "AttributeType": "S" + }, + { + "AttributeName": "tablePath", "AttributeType": "S" } ], - "TableName": "delta_rs_lock_table", + "TableName": "delta_log", "KeySchema": [ { - "AttributeName": "key", + "AttributeName": "tablePath", "KeyType": "HASH" + }, + { + "AttributeName": "fileName", + "KeyType": "RANGE" } - ] - } + ], + } Here is an example writing to s3 using this mechanism: @@ -523,16 +532,37 @@ Here is an example writing to s3 using this mechanism: >>> from deltalake import write_deltalake >>> df = pd.DataFrame({'x': [1, 2, 3]}) - >>> storage_options = {'AWS_S3_LOCKING_PROVIDER': 'dynamodb', 'DYNAMO_LOCK_TABLE_NAME': 'custom_table_name'} - >>> write_deltalake('s3://path/to/table', df, 'storage_options'= storage_options) + >>> storage_options = {'AWS_S3_LOCKING_PROVIDER': 'dynamodb', 'DELTA_DYNAMO_TABLE_NAME': 'custom_table_name'} + >>> write_deltalake('s3a://path/to/table', df, 'storage_options'= storage_options) + +.. note:: + This locking mechanism is compatible with the one used by Apache Spark. The `tablePath` property, + denoting the root url of the delta table itself, is part of the primary key, and all writers + intending to write to the same table must match this property precisely. In Spark, S3 URLs + are prefixed with `s3a://`, and a table in delta-rs must be configured accordingly. + +The following code allows creating the necessary table from the AWS cli: + +.. code-block:: sh + + aws dynamodb create-table \ + --table-name delta_log \ + --attribute-definitions AttributeName=tablePath,AttributeType=S AttributeName=fileName,AttributeType=S \ + --key-schema AttributeName=tablePath,KeyType=HASH AttributeName=fileName,KeyType=RANGE \ + --provisioned-throughput ReadCapacityUnits=5,WriteCapacityUnits=5 + +You can find additional information in the `delta-rs-documentation +https://docs.delta.io/latest/delta-storage.html#multi-cluster-setup`_, which +also includes recommendations on configuring a time-to-live (TTL) for the table to +avoid growing the table indefinitely. + +https://docs.delta.io/latest/delta-storage.html#production-configuration-s3-multi-cluster .. note:: if for some reason you don't want to use dynamodb as your locking mechanism you can choose to set the `AWS_S3_ALLOW_UNSAFE_RENAME` variable to ``true`` in order to enable S3 unsafe writes. -Please note that this locking mechanism is not compatible with any other -locking mechanisms, including the one used by Spark. Updating Delta Tables --------------------- @@ -561,7 +591,7 @@ Update all the rows for the column "processed" to the value True. :meth:`DeltaTable.update` predicates and updates are all in string format. The predicates and expressions, are parsed into Apache Datafusion expressions. -Apply a soft deletion based on a predicate, so update all the rows for the column "deleted" to the value +Apply a soft deletion based on a predicate, so update all the rows for the column "deleted" to the value True where x = 3 .. code-block:: python diff --git a/python/pyproject.toml b/python/pyproject.toml index 6ffe4ca14c..9b74760948 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -18,7 +18,8 @@ classifiers = [ "Programming Language :: Python :: 3.12" ] dependencies = [ - "pyarrow>=8" + "pyarrow>=8", + "pyarrow-hotfix", ] [project.optional-dependencies] @@ -26,8 +27,8 @@ pandas = [ "pandas" ] devel = [ - "mypy", - "ruff>=0.1.5", + "mypy~=1.8.0", + "ruff~=0.3.0", "packaging>=20", "pytest", "pytest-mock", @@ -42,7 +43,7 @@ devel = [ pyspark = [ "pyspark", "delta-spark", - "numpy==1.22.2" # pyspark is no compatible with latest numpy + "numpy==1.22.2" # pyspark is not compatible with latest numpy ] [project.urls] @@ -70,7 +71,6 @@ warn_return_any = false implicit_reexport = true strict_equality = true - [tool.black] include = '\.pyi?$' exclude = "venv" @@ -101,4 +101,4 @@ markers = [ "azure: marks tests as integration tests with Azure Blob Store", "pandas: marks tests that require pandas", "pyspark: marks tests that require pyspark", -] +] \ No newline at end of file diff --git a/python/src/error.rs b/python/src/error.rs index f72c6361d2..a54b1e60b4 100644 --- a/python/src/error.rs +++ b/python/src/error.rs @@ -10,6 +10,7 @@ create_exception!(_internal, DeltaError, PyException); create_exception!(_internal, TableNotFoundError, DeltaError); create_exception!(_internal, DeltaProtocolError, DeltaError); create_exception!(_internal, CommitFailedError, DeltaError); +create_exception!(_internal, SchemaMismatchError, DeltaError); fn inner_to_py_err(err: DeltaTableError) -> PyErr { match err { @@ -20,7 +21,7 @@ fn inner_to_py_err(err: DeltaTableError) -> PyErr { DeltaTableError::InvalidJsonLog { .. } => DeltaProtocolError::new_err(err.to_string()), DeltaTableError::InvalidStatsJson { .. } => DeltaProtocolError::new_err(err.to_string()), DeltaTableError::InvalidData { violations } => { - DeltaProtocolError::new_err(format!("Inaviant violations: {:?}", violations)) + DeltaProtocolError::new_err(format!("Invariant violations: {:?}", violations)) } // commit errors @@ -55,6 +56,7 @@ fn arrow_to_py(err: ArrowError) -> PyErr { ArrowError::DivideByZero => PyValueError::new_err("division by zero"), ArrowError::InvalidArgumentError(msg) => PyValueError::new_err(msg), ArrowError::NotYetImplemented(msg) => PyNotImplementedError::new_err(msg), + ArrowError::SchemaError(msg) => SchemaMismatchError::new_err(msg), other => PyException::new_err(other.to_string()), } } diff --git a/python/src/lib.rs b/python/src/lib.rs index 95957dd32f..445ef921ea 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -13,7 +13,6 @@ use std::time; use std::time::{SystemTime, UNIX_EPOCH}; use arrow::pyarrow::PyArrowType; -use arrow_schema::DataType; use chrono::{DateTime, Duration, FixedOffset, Utc}; use deltalake::arrow::compute::concat_batches; use deltalake::arrow::ffi_stream::ArrowArrayStreamReader; @@ -26,24 +25,28 @@ use deltalake::datafusion::datasource::provider::TableProvider; use deltalake::datafusion::prelude::SessionContext; use deltalake::delta_datafusion::DeltaDataChecker; use deltalake::errors::DeltaTableError; -use deltalake::kernel::{Action, Add, Invariant, Remove, StructType}; +use deltalake::kernel::{Action, Add, Invariant, LogicalFile, Remove, Scalar, StructType}; +use deltalake::operations::constraints::ConstraintBuilder; use deltalake::operations::convert_to_delta::{ConvertToDeltaBuilder, PartitionStrategy}; use deltalake::operations::delete::DeleteBuilder; +use deltalake::operations::drop_constraints::DropConstraintBuilder; use deltalake::operations::filesystem_check::FileSystemCheckBuilder; use deltalake::operations::merge::MergeBuilder; use deltalake::operations::optimize::{OptimizeBuilder, OptimizeType}; use deltalake::operations::restore::RestoreBuilder; -use deltalake::operations::transaction::commit; +use deltalake::operations::transaction::{CommitBuilder, CommitProperties}; use deltalake::operations::update::UpdateBuilder; use deltalake::operations::vacuum::VacuumBuilder; +use deltalake::parquet::basic::Compression; +use deltalake::parquet::errors::ParquetError; use deltalake::parquet::file::properties::WriterProperties; use deltalake::partitions::PartitionFilter; -use deltalake::protocol::{ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats}; -use deltalake::DeltaOps; +use deltalake::protocol::{DeltaOperation, SaveMode}; use deltalake::DeltaTableBuilder; +use deltalake::{DeltaOps, DeltaResult}; use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::prelude::*; -use pyo3::types::PyFrozenSet; +use pyo3::types::{PyDict, PyFrozenSet}; use serde_json::{Map, Value}; use crate::error::DeltaProtocolError; @@ -85,6 +88,8 @@ struct RawDeltaTableMetaData { configuration: HashMap>, } +type StringVec = Vec; + #[pymethods] impl RawDeltaTable { #[new] @@ -143,10 +148,35 @@ impl RawDeltaTable { }) } - pub fn protocol_versions(&self) -> PyResult<(i32, i32)> { + pub fn protocol_versions(&self) -> PyResult<(i32, i32, Option, Option)> { + let table_protocol = self._table.protocol().map_err(PythonError::from)?; Ok(( - self._table.protocol().min_reader_version, - self._table.protocol().min_writer_version, + table_protocol.min_reader_version, + table_protocol.min_writer_version, + table_protocol + .writer_features + .as_ref() + .and_then(|features| { + let empty_set = !features.is_empty(); + empty_set.then(|| { + features + .iter() + .map(|v| v.to_string()) + .collect::>() + }) + }), + table_protocol + .reader_features + .as_ref() + .and_then(|features| { + let empty_set = !features.is_empty(); + empty_set.then(|| { + features + .iter() + .map(|v| v.to_string()) + .collect::>() + }) + }), )) } @@ -216,6 +246,7 @@ impl RawDeltaTable { Ok(self ._table .get_files_iter() + .map_err(PythonError::from)? .map(|f| f.to_string()) .collect()) } @@ -232,7 +263,11 @@ impl RawDeltaTable { .get_file_uris_by_partitions(&filters) .map_err(PythonError::from)?) } else { - Ok(self._table.get_file_uris().collect()) + Ok(self + ._table + .get_file_uris() + .map_err(PythonError::from)? + .collect()) } } @@ -244,19 +279,31 @@ impl RawDeltaTable { /// Run the Vacuum command on the Delta Table: list and delete files no longer referenced /// by the Delta table and are older than the retention threshold. - #[pyo3(signature = (dry_run, retention_hours = None, enforce_retention_duration = true))] + #[pyo3(signature = (dry_run, retention_hours = None, enforce_retention_duration = true, custom_metadata=None))] pub fn vacuum( &mut self, dry_run: bool, retention_hours: Option, enforce_retention_duration: bool, + custom_metadata: Option>, ) -> PyResult> { - let mut cmd = VacuumBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_enforce_retention_duration(enforce_retention_duration) - .with_dry_run(dry_run); + let mut cmd = VacuumBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_enforce_retention_duration(enforce_retention_duration) + .with_dry_run(dry_run); if let Some(retention_period) = retention_hours { cmd = cmd.with_retention_period(Duration::hours(retention_period as i64)); } + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + let (table, metrics) = rt()? .block_on(cmd.into_future()) .map_err(PythonError::from)?; @@ -265,41 +312,25 @@ impl RawDeltaTable { } /// Run the UPDATE command on the Delta Table - #[pyo3(signature = (updates, predicate=None, writer_properties=None, safe_cast = false))] + #[pyo3(signature = (updates, predicate=None, writer_properties=None, safe_cast = false, custom_metadata = None))] pub fn update( &mut self, updates: HashMap, predicate: Option, - writer_properties: Option>, + writer_properties: Option>>, safe_cast: bool, + custom_metadata: Option>, ) -> PyResult { - let mut cmd = UpdateBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_safe_cast(safe_cast); + let mut cmd = UpdateBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_safe_cast(safe_cast); if let Some(writer_props) = writer_properties { - let mut properties = WriterProperties::builder(); - let data_page_size_limit = writer_props.get("data_page_size_limit"); - let dictionary_page_size_limit = writer_props.get("dictionary_page_size_limit"); - let data_page_row_count_limit = writer_props.get("data_page_row_count_limit"); - let write_batch_size = writer_props.get("write_batch_size"); - let max_row_group_size = writer_props.get("max_row_group_size"); - - if let Some(data_page_size) = data_page_size_limit { - properties = properties.set_data_page_size_limit(*data_page_size); - } - if let Some(dictionary_page_size) = dictionary_page_size_limit { - properties = properties.set_dictionary_page_size_limit(*dictionary_page_size); - } - if let Some(data_page_row_count) = data_page_row_count_limit { - properties = properties.set_data_page_row_count_limit(*data_page_row_count); - } - if let Some(batch_size) = write_batch_size { - properties = properties.set_write_batch_size(*batch_size); - } - if let Some(row_group_size) = max_row_group_size { - properties = properties.set_max_row_group_size(*row_group_size); - } - cmd = cmd.with_writer_properties(properties.build()); + cmd = cmd.with_writer_properties( + set_writer_properties(writer_props).map_err(PythonError::from)?, + ); } for (col_name, expression) in updates { @@ -310,6 +341,13 @@ impl RawDeltaTable { cmd = cmd.with_predicate(update_predicate); } + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + let (table, metrics) = rt()? .block_on(cmd.into_future()) .map_err(PythonError::from)?; @@ -318,22 +356,48 @@ impl RawDeltaTable { } /// Run the optimize command on the Delta Table: merge small files into a large file by bin-packing. - #[pyo3(signature = (partition_filters = None, target_size = None, max_concurrent_tasks = None, min_commit_interval = None))] + #[pyo3(signature = ( + partition_filters = None, + target_size = None, + max_concurrent_tasks = None, + min_commit_interval = None, + writer_properties=None, + custom_metadata=None, + ))] pub fn compact_optimize( &mut self, partition_filters: Option>, target_size: Option, max_concurrent_tasks: Option, min_commit_interval: Option, + writer_properties: Option>>, + custom_metadata: Option>, ) -> PyResult { - let mut cmd = OptimizeBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)); + let mut cmd = OptimizeBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)); if let Some(size) = target_size { cmd = cmd.with_target_size(size); } if let Some(commit_interval) = min_commit_interval { cmd = cmd.with_min_commit_interval(time::Duration::from_secs(commit_interval)); } + + if let Some(writer_props) = writer_properties { + cmd = cmd.with_writer_properties( + set_writer_properties(writer_props).map_err(PythonError::from)?, + ); + } + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + let converted_filters = convert_partition_filters(partition_filters.unwrap_or_default()) .map_err(PythonError::from)?; cmd = cmd.with_filters(&converted_filters); @@ -346,7 +410,15 @@ impl RawDeltaTable { } /// Run z-order variation of optimize - #[pyo3(signature = (z_order_columns, partition_filters = None, target_size = None, max_concurrent_tasks = None, max_spill_size = 20 * 1024 * 1024 * 1024, min_commit_interval = None))] + #[allow(clippy::too_many_arguments)] + #[pyo3(signature = (z_order_columns, + partition_filters = None, + target_size = None, + max_concurrent_tasks = None, + max_spill_size = 20 * 1024 * 1024 * 1024, + min_commit_interval = None, + writer_properties=None, + custom_metadata=None,))] pub fn z_order_optimize( &mut self, z_order_columns: Vec, @@ -355,11 +427,16 @@ impl RawDeltaTable { max_concurrent_tasks: Option, max_spill_size: usize, min_commit_interval: Option, + writer_properties: Option>>, + custom_metadata: Option>, ) -> PyResult { - let mut cmd = OptimizeBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)) - .with_max_spill_size(max_spill_size) - .with_type(OptimizeType::ZOrder(z_order_columns)); + let mut cmd = OptimizeBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)) + .with_max_spill_size(max_spill_size) + .with_type(OptimizeType::ZOrder(z_order_columns)); if let Some(size) = target_size { cmd = cmd.with_target_size(size); } @@ -367,6 +444,19 @@ impl RawDeltaTable { cmd = cmd.with_min_commit_interval(time::Duration::from_secs(commit_interval)); } + if let Some(writer_props) = writer_properties { + cmd = cmd.with_writer_properties( + set_writer_properties(writer_props).map_err(PythonError::from)?, + ); + } + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + let converted_filters = convert_partition_filters(partition_filters.unwrap_or_default()) .map_err(PythonError::from)?; cmd = cmd.with_filters(&converted_filters); @@ -378,6 +468,63 @@ impl RawDeltaTable { Ok(serde_json::to_string(&metrics).unwrap()) } + #[pyo3(signature = (constraints, custom_metadata=None))] + pub fn add_constraints( + &mut self, + constraints: HashMap, + custom_metadata: Option>, + ) -> PyResult<()> { + let mut cmd = ConstraintBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ); + + for (col_name, expression) in constraints { + cmd = cmd.with_constraint(col_name.clone(), expression.clone()); + } + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + + let table = rt()? + .block_on(cmd.into_future()) + .map_err(PythonError::from)?; + self._table.state = table.state; + Ok(()) + } + + #[pyo3(signature = (name, raise_if_not_exists, custom_metadata=None))] + pub fn drop_constraints( + &mut self, + name: String, + raise_if_not_exists: bool, + custom_metadata: Option>, + ) -> PyResult<()> { + let mut cmd = DropConstraintBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_constraint(name) + .with_raise_if_not_exists(raise_if_not_exists); + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + + let table = rt()? + .block_on(cmd.into_future()) + .map_err(PythonError::from)?; + self._table.state = table.state; + Ok(()) + } + #[allow(clippy::too_many_arguments)] #[pyo3(signature = (source, predicate, @@ -385,6 +532,7 @@ impl RawDeltaTable { target_alias = None, safe_cast = false, writer_properties = None, + custom_metadata = None, matched_update_updates = None, matched_update_predicate = None, matched_delete_predicate = None, @@ -398,12 +546,14 @@ impl RawDeltaTable { ))] pub fn merge_execute( &mut self, + py: Python, source: PyArrowType, predicate: String, source_alias: Option, target_alias: Option, safe_cast: bool, - writer_properties: Option>, + writer_properties: Option>>, + custom_metadata: Option>, matched_update_updates: Option>>, matched_update_predicate: Option>>, matched_delete_predicate: Option>, @@ -415,177 +565,175 @@ impl RawDeltaTable { not_matched_by_source_delete_predicate: Option>, not_matched_by_source_delete_all: Option, ) -> PyResult { - let ctx = SessionContext::new(); - let schema = source.0.schema(); - let batches = vec![source.0.map(|batch| batch.unwrap()).collect::>()]; - let table_provider: Arc = - Arc::new(MemTable::try_new(schema, batches).unwrap()); - let source_df = ctx.read_table(table_provider).unwrap(); - - let mut cmd = MergeBuilder::new( - self._table.log_store(), - self._table.state.clone(), - predicate, - source_df, - ) - .with_safe_cast(safe_cast); - - if let Some(src_alias) = source_alias { - cmd = cmd.with_source_alias(src_alias); - } - - if let Some(trgt_alias) = target_alias { - cmd = cmd.with_target_alias(trgt_alias); - } + py.allow_threads(|| { + let ctx = SessionContext::new(); + let schema = source.0.schema(); + let batches = vec![source.0.map(|batch| batch.unwrap()).collect::>()]; + let table_provider: Arc = + Arc::new(MemTable::try_new(schema, batches).unwrap()); + let source_df = ctx.read_table(table_provider).unwrap(); + + let mut cmd = MergeBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + predicate, + source_df, + ) + .with_safe_cast(safe_cast); - if let Some(writer_props) = writer_properties { - let mut properties = WriterProperties::builder(); - let data_page_size_limit = writer_props.get("data_page_size_limit"); - let dictionary_page_size_limit = writer_props.get("dictionary_page_size_limit"); - let data_page_row_count_limit = writer_props.get("data_page_row_count_limit"); - let write_batch_size = writer_props.get("write_batch_size"); - let max_row_group_size = writer_props.get("max_row_group_size"); - - if let Some(data_page_size) = data_page_size_limit { - properties = properties.set_data_page_size_limit(*data_page_size); - } - if let Some(dictionary_page_size) = dictionary_page_size_limit { - properties = properties.set_dictionary_page_size_limit(*dictionary_page_size); - } - if let Some(data_page_row_count) = data_page_row_count_limit { - properties = properties.set_data_page_row_count_limit(*data_page_row_count); + if let Some(src_alias) = source_alias { + cmd = cmd.with_source_alias(src_alias); } - if let Some(batch_size) = write_batch_size { - properties = properties.set_write_batch_size(*batch_size); + + if let Some(trgt_alias) = target_alias { + cmd = cmd.with_target_alias(trgt_alias); } - if let Some(row_group_size) = max_row_group_size { - properties = properties.set_max_row_group_size(*row_group_size); + + if let Some(writer_props) = writer_properties { + cmd = cmd.with_writer_properties( + set_writer_properties(writer_props).map_err(PythonError::from)?, + ); } - cmd = cmd.with_writer_properties(properties.build()); - } - if let Some(mu_updates) = matched_update_updates { - if let Some(mu_predicate) = matched_update_predicate { - for it in mu_updates.iter().zip(mu_predicate.iter()) { - let (update_values, predicate_value) = it; - - if let Some(pred) = predicate_value { - cmd = cmd - .when_matched_update(|mut update| { - for (col_name, expression) in update_values { - update = update.update(col_name.clone(), expression.clone()); - } - update.predicate(pred.clone()) - }) - .map_err(PythonError::from)?; - } else { - cmd = cmd - .when_matched_update(|mut update| { - for (col_name, expression) in update_values { - update = update.update(col_name.clone(), expression.clone()); - } - update - }) - .map_err(PythonError::from)?; + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd.with_commit_properties( + CommitProperties::default().with_metadata(json_metadata), + ); + }; + + if let Some(mu_updates) = matched_update_updates { + if let Some(mu_predicate) = matched_update_predicate { + for it in mu_updates.iter().zip(mu_predicate.iter()) { + let (update_values, predicate_value) = it; + + if let Some(pred) = predicate_value { + cmd = cmd + .when_matched_update(|mut update| { + for (col_name, expression) in update_values { + update = + update.update(col_name.clone(), expression.clone()); + } + update.predicate(pred.clone()) + }) + .map_err(PythonError::from)?; + } else { + cmd = cmd + .when_matched_update(|mut update| { + for (col_name, expression) in update_values { + update = + update.update(col_name.clone(), expression.clone()); + } + update + }) + .map_err(PythonError::from)?; + } } } } - } - if let Some(_md_delete_all) = matched_delete_all { - cmd = cmd - .when_matched_delete(|delete| delete) - .map_err(PythonError::from)?; - } else if let Some(md_predicate) = matched_delete_predicate { - for pred in md_predicate.iter() { + if let Some(_md_delete_all) = matched_delete_all { cmd = cmd - .when_matched_delete(|delete| delete.predicate(pred.clone())) + .when_matched_delete(|delete| delete) .map_err(PythonError::from)?; + } else if let Some(md_predicate) = matched_delete_predicate { + for pred in md_predicate.iter() { + cmd = cmd + .when_matched_delete(|delete| delete.predicate(pred.clone())) + .map_err(PythonError::from)?; + } } - } - if let Some(nmi_updates) = not_matched_insert_updates { - if let Some(nmi_predicate) = not_matched_insert_predicate { - for it in nmi_updates.iter().zip(nmi_predicate.iter()) { - let (update_values, predicate_value) = it; - if let Some(pred) = predicate_value { - cmd = cmd - .when_not_matched_insert(|mut insert| { - for (col_name, expression) in update_values { - insert = insert.set(col_name.clone(), expression.clone()); - } - insert.predicate(pred.clone()) - }) - .map_err(PythonError::from)?; - } else { - cmd = cmd - .when_not_matched_insert(|mut insert| { - for (col_name, expression) in update_values { - insert = insert.set(col_name.clone(), expression.clone()); - } - insert - }) - .map_err(PythonError::from)?; + if let Some(nmi_updates) = not_matched_insert_updates { + if let Some(nmi_predicate) = not_matched_insert_predicate { + for it in nmi_updates.iter().zip(nmi_predicate.iter()) { + let (update_values, predicate_value) = it; + if let Some(pred) = predicate_value { + cmd = cmd + .when_not_matched_insert(|mut insert| { + for (col_name, expression) in update_values { + insert = insert.set(col_name.clone(), expression.clone()); + } + insert.predicate(pred.clone()) + }) + .map_err(PythonError::from)?; + } else { + cmd = cmd + .when_not_matched_insert(|mut insert| { + for (col_name, expression) in update_values { + insert = insert.set(col_name.clone(), expression.clone()); + } + insert + }) + .map_err(PythonError::from)?; + } } } } - } - if let Some(nmbsu_updates) = not_matched_by_source_update_updates { - if let Some(nmbsu_predicate) = not_matched_by_source_update_predicate { - for it in nmbsu_updates.iter().zip(nmbsu_predicate.iter()) { - let (update_values, predicate_value) = it; - if let Some(pred) = predicate_value { - cmd = cmd - .when_not_matched_by_source_update(|mut update| { - for (col_name, expression) in update_values { - update = update.update(col_name.clone(), expression.clone()); - } - update.predicate(pred.clone()) - }) - .map_err(PythonError::from)?; - } else { - cmd = cmd - .when_not_matched_by_source_update(|mut update| { - for (col_name, expression) in update_values { - update = update.update(col_name.clone(), expression.clone()); - } - update - }) - .map_err(PythonError::from)?; + if let Some(nmbsu_updates) = not_matched_by_source_update_updates { + if let Some(nmbsu_predicate) = not_matched_by_source_update_predicate { + for it in nmbsu_updates.iter().zip(nmbsu_predicate.iter()) { + let (update_values, predicate_value) = it; + if let Some(pred) = predicate_value { + cmd = cmd + .when_not_matched_by_source_update(|mut update| { + for (col_name, expression) in update_values { + update = + update.update(col_name.clone(), expression.clone()); + } + update.predicate(pred.clone()) + }) + .map_err(PythonError::from)?; + } else { + cmd = cmd + .when_not_matched_by_source_update(|mut update| { + for (col_name, expression) in update_values { + update = + update.update(col_name.clone(), expression.clone()); + } + update + }) + .map_err(PythonError::from)?; + } } } } - } - if let Some(_nmbs_delete_all) = not_matched_by_source_delete_all { - cmd = cmd - .when_not_matched_by_source_delete(|delete| delete) - .map_err(PythonError::from)?; - } else if let Some(nmbs_predicate) = not_matched_by_source_delete_predicate { - for pred in nmbs_predicate.iter() { + if let Some(_nmbs_delete_all) = not_matched_by_source_delete_all { cmd = cmd - .when_not_matched_by_source_delete(|delete| delete.predicate(pred.clone())) + .when_not_matched_by_source_delete(|delete| delete) .map_err(PythonError::from)?; + } else if let Some(nmbs_predicate) = not_matched_by_source_delete_predicate { + for pred in nmbs_predicate.iter() { + cmd = cmd + .when_not_matched_by_source_delete(|delete| delete.predicate(pred.clone())) + .map_err(PythonError::from)?; + } } - } - let (table, metrics) = rt()? - .block_on(cmd.into_future()) - .map_err(PythonError::from)?; - self._table.state = table.state; - Ok(serde_json::to_string(&metrics).unwrap()) + let (table, metrics) = rt()? + .block_on(cmd.into_future()) + .map_err(PythonError::from)?; + self._table.state = table.state; + Ok(serde_json::to_string(&metrics).unwrap()) + }) } // Run the restore command on the Delta Table: restore table to a given version or datetime - #[pyo3(signature = (target, *, ignore_missing_files = false, protocol_downgrade_allowed = false))] + #[pyo3(signature = (target, *, ignore_missing_files = false, protocol_downgrade_allowed = false, custom_metadata=None))] pub fn restore( &mut self, target: Option<&PyAny>, ignore_missing_files: bool, protocol_downgrade_allowed: bool, + custom_metadata: Option>, ) -> PyResult { - let mut cmd = RestoreBuilder::new(self._table.log_store(), self._table.state.clone()); + let mut cmd = RestoreBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ); if let Some(val) = target { if let Ok(version) = val.extract::() { cmd = cmd.with_version_to_restore(version) @@ -601,6 +749,14 @@ impl RawDeltaTable { } cmd = cmd.with_ignore_missing_files(ignore_missing_files); cmd = cmd.with_protocol_downgrade_allowed(protocol_downgrade_allowed); + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + let (table, metrics) = rt()? .block_on(cmd.into_future()) .map_err(PythonError::from)?; @@ -620,6 +776,7 @@ impl RawDeltaTable { } pub fn update_incremental(&mut self) -> PyResult<()> { + #[allow(deprecated)] Ok(rt()? .block_on(self._table.update_incremental(None)) .map_err(PythonError::from)?) @@ -637,19 +794,20 @@ impl RawDeltaTable { )), None => None, }; - self._table - .get_files_iter() - .map(|p| p.to_string()) - .zip(self._table.get_partition_values()) - .zip(self._table.get_stats()) - .filter(|((path, _), _)| match &path_set { - Some(path_set) => path_set.contains(path), - None => true, + .snapshot() + .map_err(PythonError::from)? + .log_data() + .into_iter() + .filter_map(|f| { + let path = f.path().to_string(); + match &path_set { + Some(path_set) => path_set.contains(&path).then_some((path, f)), + None => Some((path, f)), + } }) - .map(|((path, partition_values), stats)| { - let stats = stats.map_err(PythonError::from)?; - let expression = filestats_to_expression(py, &schema, partition_values, stats)?; + .map(|(path, f)| { + let expression = filestats_to_expression_next(py, &schema, f)?; Ok((path, expression)) }) .collect() @@ -707,16 +865,31 @@ impl RawDeltaTable { let partition_columns: Vec<&str> = partition_columns.into_iter().collect(); - let active_partitions: HashSet)>> = self + let adds = self ._table - .get_state() + .snapshot() + .map_err(PythonError::from)? .get_active_add_actions_by_partitions(&converted_filters) .map_err(PythonError::from)? - .map(|add| { - partition_columns - .iter() - .map(|col| (*col, add.partition_values.get(*col).unwrap().as_deref())) - .collect() + .collect::, _>>() + .map_err(PythonError::from)?; + let active_partitions: HashSet)>> = adds + .iter() + .flat_map(|add| { + Ok::<_, PythonError>( + partition_columns + .iter() + .flat_map(|col| { + Ok::<_, PythonError>(( + *col, + add.partition_values() + .map_err(PythonError::from)? + .get(*col) + .map(|v| v.serialize()), + )) + }) + .collect(), + ) }) .collect(); @@ -734,6 +907,7 @@ impl RawDeltaTable { partition_by: Vec, schema: PyArrowType, partitions_filters: Option>, + custom_metadata: Option>, ) -> PyResult<()> { let mode = mode.parse().map_err(PythonError::from)?; @@ -754,22 +928,40 @@ impl RawDeltaTable { let add_actions = self ._table - .get_state() + .snapshot() + .map_err(PythonError::from)? .get_active_add_actions_by_partitions(&converted_filters) .map_err(PythonError::from)?; for old_add in add_actions { + let old_add = old_add.map_err(PythonError::from)?; let remove_action = Action::Remove(Remove { - path: old_add.path.clone(), + path: old_add.path().to_string(), deletion_timestamp: Some(current_timestamp()), data_change: true, - extended_file_metadata: Some(old_add.tags.is_some()), - partition_values: Some(old_add.partition_values.clone()), - size: Some(old_add.size), - deletion_vector: old_add.deletion_vector.clone(), - tags: old_add.tags.clone(), - base_row_id: old_add.base_row_id, - default_row_commit_version: old_add.default_row_commit_version, + extended_file_metadata: Some(true), + partition_values: Some( + old_add + .partition_values() + .map_err(PythonError::from)? + .iter() + .map(|(k, v)| { + ( + k.to_string(), + if v.is_null() { + None + } else { + Some(v.serialize()) + }, + ) + }) + .collect(), + ), + size: Some(old_add.size()), + deletion_vector: None, + tags: None, + base_row_id: None, + default_row_commit_version: None, }); actions.push(remove_action); } @@ -796,16 +988,26 @@ impl RawDeltaTable { partition_by: Some(partition_by), predicate: None, }; - let store = self._table.log_store(); rt()? - .block_on(commit( - &*store, - &actions, - operation, - self._table.get_state(), - None, - )) + .block_on( + CommitBuilder::from( + CommitProperties::default().with_metadata( + custom_metadata + .unwrap_or_default() + .into_iter() + .map(|(k, v)| (k, v.into())), + ), + ) + .with_actions(actions) + .build( + Some(self._table.snapshot().map_err(PythonError::from)?), + self._table.log_store(), + operation, + ) + .map_err(|err| PythonError::from(DeltaTableError::from(err)))? + .into_future(), + ) .map_err(PythonError::from)?; Ok(()) @@ -839,19 +1041,42 @@ impl RawDeltaTable { pub fn get_add_actions(&self, flatten: bool) -> PyResult> { Ok(PyArrowType( self._table - .get_state() + .snapshot() + .map_err(PythonError::from)? .add_actions_table(flatten) .map_err(PythonError::from)?, )) } /// Run the delete command on the delta table: delete records following a predicate and return the delete metrics. - #[pyo3(signature = (predicate = None))] - pub fn delete(&mut self, predicate: Option) -> PyResult { - let mut cmd = DeleteBuilder::new(self._table.log_store(), self._table.state.clone()); + #[pyo3(signature = (predicate = None, writer_properties=None, custom_metadata=None))] + pub fn delete( + &mut self, + predicate: Option, + writer_properties: Option>>, + custom_metadata: Option>, + ) -> PyResult { + let mut cmd = DeleteBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ); if let Some(predicate) = predicate { cmd = cmd.with_predicate(predicate); } + + if let Some(writer_props) = writer_properties { + cmd = cmd.with_writer_properties( + set_writer_properties(writer_props).map_err(PythonError::from)?, + ); + } + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; + let (table, metrics) = rt()? .block_on(cmd.into_future()) .map_err(PythonError::from)?; @@ -861,10 +1086,24 @@ impl RawDeltaTable { /// Execute the File System Check command (FSCK) on the delta table: removes old reference to files that /// have been deleted or are malformed - #[pyo3(signature = (dry_run = true))] - pub fn repair(&mut self, dry_run: bool) -> PyResult { - let cmd = FileSystemCheckBuilder::new(self._table.log_store(), self._table.state.clone()) - .with_dry_run(dry_run); + #[pyo3(signature = (dry_run = true, custom_metadata = None))] + pub fn repair( + &mut self, + dry_run: bool, + custom_metadata: Option>, + ) -> PyResult { + let mut cmd = FileSystemCheckBuilder::new( + self._table.log_store(), + self._table.snapshot().map_err(PythonError::from)?.clone(), + ) + .with_dry_run(dry_run); + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + cmd = cmd + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; let (table, metrics) = rt()? .block_on(cmd.into_future()) @@ -874,6 +1113,45 @@ impl RawDeltaTable { } } +fn set_writer_properties( + writer_properties: HashMap>, +) -> DeltaResult { + let mut properties = WriterProperties::builder(); + let data_page_size_limit = writer_properties.get("data_page_size_limit"); + let dictionary_page_size_limit = writer_properties.get("dictionary_page_size_limit"); + let data_page_row_count_limit = writer_properties.get("data_page_row_count_limit"); + let write_batch_size = writer_properties.get("write_batch_size"); + let max_row_group_size = writer_properties.get("max_row_group_size"); + let compression = writer_properties.get("compression"); + + if let Some(Some(data_page_size)) = data_page_size_limit { + properties = properties.set_data_page_size_limit(data_page_size.parse::().unwrap()); + } + if let Some(Some(dictionary_page_size)) = dictionary_page_size_limit { + properties = properties + .set_dictionary_page_size_limit(dictionary_page_size.parse::().unwrap()); + } + if let Some(Some(data_page_row_count)) = data_page_row_count_limit { + properties = + properties.set_data_page_row_count_limit(data_page_row_count.parse::().unwrap()); + } + if let Some(Some(batch_size)) = write_batch_size { + properties = properties.set_write_batch_size(batch_size.parse::().unwrap()); + } + if let Some(Some(row_group_size)) = max_row_group_size { + properties = properties.set_max_row_group_size(row_group_size.parse::().unwrap()); + } + + if let Some(Some(compression)) = compression { + let compress: Compression = compression + .parse() + .map_err(|err: ParquetError| DeltaTableError::Generic(err.to_string()))?; + + properties = properties.set_compression(compress); + } + Ok(properties.build()) +} + fn convert_partition_filters<'a>( partitions_filters: Vec<(&'a str, &'a str, PartitionFilterValue)>, ) -> Result, DeltaTableError> { @@ -888,22 +1166,47 @@ fn convert_partition_filters<'a>( .collect() } -fn json_value_to_py(value: &serde_json::Value, py: Python) -> PyObject { - match value { - serde_json::Value::Null => py.None(), - serde_json::Value::Bool(val) => val.to_object(py), - serde_json::Value::Number(val) => { - if val.is_f64() { - val.as_f64().expect("not an f64").to_object(py) - } else if val.is_i64() { - val.as_i64().expect("not an i64").to_object(py) - } else { - val.as_u64().expect("not an u64").to_object(py) +fn scalar_to_py(value: &Scalar, py_date: &PyAny, py: Python) -> PyResult { + use Scalar::*; + + let val = match value { + Null(_) => py.None(), + Boolean(val) => val.to_object(py), + Binary(val) => val.to_object(py), + String(val) => val.to_object(py), + Byte(val) => val.to_object(py), + Short(val) => val.to_object(py), + Integer(val) => val.to_object(py), + Long(val) => val.to_object(py), + Float(val) => val.to_object(py), + Double(val) => val.to_object(py), + Timestamp(_) => { + // We need to manually append 'Z' add to end so that pyarrow can cast the + // the scalar value to pa.timestamp("us","UTC") + let value = value.serialize(); + format!("{}Z", value).to_object(py) + } + TimestampNtz(_) => { + let value = value.serialize(); + value.to_object(py) + } + // NOTE: PyArrow 13.0.0 lost the ability to cast from string to date32, so + // we have to implement that manually. + Date(_) => { + let date = py_date.call_method1("fromisoformat", (value.serialize(),))?; + date.to_object(py) + } + Decimal(_, _, _) => value.serialize().to_object(py), + Struct(values, fields) => { + let py_struct = PyDict::new(py); + for (field, value) in fields.iter().zip(values.iter()) { + py_struct.set_item(field.name(), scalar_to_py(value, py_date, py)?)?; } + py_struct.to_object(py) } - serde_json::Value::String(val) => val.to_object(py), - _ => py.None(), - } + }; + + Ok(val) } /// Create expression that file statistics guarantee to be true. @@ -918,15 +1221,15 @@ fn json_value_to_py(value: &serde_json::Value, py: Python) -> PyObject { /// /// Statistics are translated into inequalities. If there are null values, then /// they must be OR'd with is_null. -fn filestats_to_expression<'py>( +fn filestats_to_expression_next<'py>( py: Python<'py>, schema: &PyArrowType, - partitions_values: &HashMap>, - stats: Option, + file_info: LogicalFile<'_>, ) -> PyResult> { let ds = PyModule::import(py, "pyarrow.dataset")?; - let field = ds.getattr("field")?; + let py_field = ds.getattr("field")?; let pa = PyModule::import(py, "pyarrow")?; + let py_date = Python::import(py, "datetime")?.getattr("date")?; let mut expressions: Vec> = Vec::new(); let cast_to_type = |column_name: &String, value: PyObject, schema: &ArrowSchema| { @@ -937,97 +1240,96 @@ fn filestats_to_expression<'py>( })? .data_type() .clone(); - - let value = match column_type { - // Since PyArrow 13.0.0, casting string -> timestamp fails if it ends with "Z" - // and the target type is timezone naive. - DataType::Timestamp(_, _) if value.extract::(py).is_ok() => { - value.call_method1(py, "rstrip", ("Z",))? - } - // PyArrow 13.0.0 lost the ability to cast from string to date32, so - // we have to implement that manually. - DataType::Date32 if value.extract::(py).is_ok() => { - let date = Python::import(py, "datetime")?.getattr("date")?; - let date = date.call_method1("fromisoformat", (value,))?; - date.to_object(py) - } - _ => value, - }; - let column_type = PyArrowType(column_type).into_py(py); pa.call_method1("scalar", (value,))? .call_method1("cast", (column_type,)) }; - for (column, value) in partitions_values.iter() { - if let Some(value) = value { - // value is a string, but needs to be parsed into appropriate type - let converted_value = cast_to_type(column, value.into_py(py), &schema.0)?; - expressions.push( - field - .call1((column,))? - .call_method1("__eq__", (converted_value,)), - ); - } else { - expressions.push(field.call1((column,))?.call_method0("is_null")); + if let Ok(partitions_values) = file_info.partition_values() { + for (column, value) in partitions_values.iter() { + let column = column.to_string(); + if !value.is_null() { + // value is a string, but needs to be parsed into appropriate type + let converted_value = + cast_to_type(&column, scalar_to_py(value, py_date, py)?, &schema.0)?; + expressions.push( + py_field + .call1((&column,))? + .call_method1("__eq__", (converted_value,)), + ); + } else { + expressions.push(py_field.call1((column,))?.call_method0("is_null")); + } } } - if let Some(stats) = stats { - let mut has_nulls_set: HashSet = HashSet::new(); - - for (col_name, null_count) in stats.null_count.iter().filter_map(|(k, v)| match v { - ColumnCountStat::Value(val) => Some((k, val)), - _ => None, - }) { - if *null_count == 0 { - expressions.push(field.call1((col_name,))?.call_method0("is_valid")); - } else if *null_count == stats.num_records { - expressions.push(field.call1((col_name,))?.call_method0("is_null")); - } else { - has_nulls_set.insert(col_name.clone()); + let mut has_nulls_set: HashSet = HashSet::new(); + + // NOTE: null_counts should always return a struct scalar. + if let Some(Scalar::Struct(values, fields)) = file_info.null_counts() { + for (field, value) in fields.iter().zip(values.iter()) { + if let Scalar::Long(val) = value { + if *val == 0 { + expressions.push(py_field.call1((field.name(),))?.call_method0("is_valid")); + } else if Some(*val as usize) == file_info.num_records() { + expressions.push(py_field.call1((field.name(),))?.call_method0("is_null")); + } else { + has_nulls_set.insert(field.name().to_string()); + } } } + } - for (col_name, minimum) in stats.min_values.iter().filter_map(|(k, v)| match v { - ColumnValueStat::Value(val) => Some((k.clone(), json_value_to_py(val, py))), - // TODO(wjones127): Handle nested field statistics. - // Blocked on https://issues.apache.org/jira/browse/ARROW-11259 - _ => None, - }) { - let maybe_minimum = cast_to_type(&col_name, minimum, &schema.0); - if let Ok(minimum) = maybe_minimum { - let field_expr = field.call1((&col_name,))?; - let expr = field_expr.call_method1("__ge__", (minimum,)); - let expr = if has_nulls_set.contains(&col_name) { - // col >= min_value OR col is null - let is_null_expr = field_expr.call_method0("is_null"); - expr?.call_method1("__or__", (is_null_expr?,)) - } else { - // col >= min_value - expr - }; - expressions.push(expr); + // NOTE: min_values should always return a struct scalar. + if let Some(Scalar::Struct(values, fields)) = file_info.min_values() { + for (field, value) in fields.iter().zip(values.iter()) { + match value { + // TODO: Handle nested field statistics. + Scalar::Struct(_, _) => {} + _ => { + let maybe_minimum = + cast_to_type(field.name(), scalar_to_py(value, py_date, py)?, &schema.0); + if let Ok(minimum) = maybe_minimum { + let field_expr = py_field.call1((field.name(),))?; + let expr = field_expr.call_method1("__ge__", (minimum,)); + let expr = if has_nulls_set.contains(field.name()) { + // col >= min_value OR col is null + let is_null_expr = field_expr.call_method0("is_null"); + expr?.call_method1("__or__", (is_null_expr?,)) + } else { + // col >= min_value + expr + }; + expressions.push(expr); + } + } } } + } - for (col_name, maximum) in stats.max_values.iter().filter_map(|(k, v)| match v { - ColumnValueStat::Value(val) => Some((k.clone(), json_value_to_py(val, py))), - _ => None, - }) { - let maybe_maximum = cast_to_type(&col_name, maximum, &schema.0); - if let Ok(maximum) = maybe_maximum { - let field_expr = field.call1((&col_name,))?; - let expr = field_expr.call_method1("__le__", (maximum,)); - let expr = if has_nulls_set.contains(&col_name) { - // col <= max_value OR col is null - let is_null_expr = field_expr.call_method0("is_null"); - expr?.call_method1("__or__", (is_null_expr?,)) - } else { - // col <= max_value - expr - }; - expressions.push(expr); + // NOTE: max_values should always return a struct scalar. + if let Some(Scalar::Struct(values, fields)) = file_info.max_values() { + for (field, value) in fields.iter().zip(values.iter()) { + match value { + // TODO: Handle nested field statistics. + Scalar::Struct(_, _) => {} + _ => { + let maybe_maximum = + cast_to_type(field.name(), scalar_to_py(value, py_date, py)?, &schema.0); + if let Ok(maximum) = maybe_maximum { + let field_expr = py_field.call1((field.name(),))?; + let expr = field_expr.call_method1("__le__", (maximum,)); + let expr = if has_nulls_set.contains(field.name()) { + // col <= max_value OR col is null + let is_null_expr = field_expr.call_method0("is_null"); + expr?.call_method1("__or__", (is_null_expr?,)) + } else { + // col <= max_value + expr + }; + expressions.push(expr); + } + } } } } @@ -1086,7 +1388,6 @@ impl From<&PyAddAction> for Add { path: action.path.clone(), size: action.size, partition_values: action.partition_values.clone(), - partition_values_parsed: None, modification_time: action.modification_time, data_change: action.data_change, stats: action.stats.clone(), @@ -1103,59 +1404,78 @@ impl From<&PyAddAction> for Add { #[pyfunction] #[allow(clippy::too_many_arguments)] fn write_to_deltalake( + py: Python, table_uri: String, data: PyArrowType, mode: String, max_rows_per_group: i64, - overwrite_schema: bool, + schema_mode: Option, partition_by: Option>, predicate: Option, name: Option, description: Option, configuration: Option>>, storage_options: Option>, + writer_properties: Option>>, + custom_metadata: Option>, ) -> PyResult<()> { - let batches = data.0.map(|batch| batch.unwrap()).collect::>(); - let save_mode = mode.parse().map_err(PythonError::from)?; + py.allow_threads(|| { + let batches = data.0.map(|batch| batch.unwrap()).collect::>(); + let save_mode = mode.parse().map_err(PythonError::from)?; - let options = storage_options.clone().unwrap_or_default(); - let table = rt()? - .block_on(DeltaOps::try_from_uri_with_storage_options( - &table_uri, options, - )) - .map_err(PythonError::from)?; + let options = storage_options.clone().unwrap_or_default(); + let table = rt()? + .block_on(DeltaOps::try_from_uri_with_storage_options( + &table_uri, options, + )) + .map_err(PythonError::from)?; - let mut builder = table - .write(batches) - .with_save_mode(save_mode) - .with_overwrite_schema(overwrite_schema) - .with_write_batch_size(max_rows_per_group as usize); + let mut builder = table + .write(batches) + .with_save_mode(save_mode) + .with_write_batch_size(max_rows_per_group as usize); + if let Some(schema_mode) = schema_mode { + builder = builder.with_schema_mode(schema_mode.parse().map_err(PythonError::from)?); + } + if let Some(partition_columns) = partition_by { + builder = builder.with_partition_columns(partition_columns); + } - if let Some(partition_columns) = partition_by { - builder = builder.with_partition_columns(partition_columns); - } + if let Some(writer_props) = writer_properties { + builder = builder.with_writer_properties( + set_writer_properties(writer_props).map_err(PythonError::from)?, + ); + } - if let Some(name) = &name { - builder = builder.with_table_name(name); - }; + if let Some(name) = &name { + builder = builder.with_table_name(name); + }; - if let Some(description) = &description { - builder = builder.with_description(description); - }; + if let Some(description) = &description { + builder = builder.with_description(description); + }; - if let Some(predicate) = &predicate { - builder = builder.with_replace_where(predicate); - }; + if let Some(predicate) = predicate { + builder = builder.with_replace_where(predicate); + }; - if let Some(config) = configuration { - builder = builder.with_configuration(config); - }; + if let Some(config) = configuration { + builder = builder.with_configuration(config); + }; - rt()? - .block_on(builder.into_future()) - .map_err(PythonError::from)?; + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + builder = builder + .with_commit_properties(CommitProperties::default().with_metadata(json_metadata)); + }; - Ok(()) + rt()? + .block_on(builder.into_future()) + .map_err(PythonError::from)?; + + Ok(()) + }) } #[pyfunction] @@ -1169,6 +1489,7 @@ fn create_deltalake( description: Option, configuration: Option>>, storage_options: Option>, + custom_metadata: Option>, ) -> PyResult<()> { let table = DeltaTableBuilder::from_uri(table_uri) .with_storage_options(storage_options.unwrap_or_default()) @@ -1196,6 +1517,12 @@ fn create_deltalake( builder = builder.with_configuration(config); }; + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + builder = builder.with_metadata(json_metadata); + }; + rt()? .block_on(builder.into_future()) .map_err(PythonError::from)?; @@ -1215,6 +1542,7 @@ fn write_new_deltalake( description: Option, configuration: Option>>, storage_options: Option>, + custom_metadata: Option>, ) -> PyResult<()> { let table = DeltaTableBuilder::from_uri(table_uri) .with_storage_options(storage_options.unwrap_or_default()) @@ -1241,6 +1569,12 @@ fn write_new_deltalake( builder = builder.with_configuration(config); }; + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + builder = builder.with_metadata(json_metadata); + }; + rt()? .block_on(builder.into_future()) .map_err(PythonError::from)?; @@ -1319,7 +1653,7 @@ impl PyDeltaDataChecker { }) .collect(); Self { - inner: DeltaDataChecker::new(invariants), + inner: DeltaDataChecker::new_with_invariants(invariants), rt: tokio::runtime::Runtime::new().unwrap(), } } @@ -1335,11 +1669,18 @@ impl PyDeltaDataChecker { #[pymodule] // module name need to match project name fn _internal(py: Python, m: &PyModule) -> PyResult<()> { - use crate::error::{CommitFailedError, DeltaError, TableNotFoundError}; + use crate::error::{CommitFailedError, DeltaError, SchemaMismatchError, TableNotFoundError}; + + deltalake::aws::register_handlers(None); + deltalake::azure::register_handlers(None); + deltalake::gcp::register_handlers(None); + deltalake::mount::register_handlers(None); + m.add("DeltaError", py.get_type::())?; m.add("CommitFailedError", py.get_type::())?; m.add("DeltaProtocolError", py.get_type::())?; m.add("TableNotFoundError", py.get_type::())?; + m.add("SchemaMismatchError", py.get_type::())?; env_logger::Builder::from_env(env_logger::Env::default().default_filter_or("warn")).init(); m.add("__version__", env!("CARGO_PKG_VERSION"))?; diff --git a/python/src/utils.rs b/python/src/utils.rs index 8d3c9d9eb3..ae4c931e38 100644 --- a/python/src/utils.rs +++ b/python/src/utils.rs @@ -74,7 +74,7 @@ fn list_with_delimiter_recursive( pub async fn delete_dir(storage: &dyn ObjectStore, prefix: &Path) -> ObjectStoreResult<()> { // TODO batch delete would be really useful now... - let mut stream = storage.list(Some(prefix)).await?; + let mut stream = storage.list(Some(prefix)); while let Some(maybe_meta) = stream.next().await { let meta = maybe_meta?; storage.delete(&meta.location).await?; diff --git a/python/stubs/pyarrow/__init__.pyi b/python/stubs/pyarrow/__init__.pyi index 10edfcf663..aaf92ea962 100644 --- a/python/stubs/pyarrow/__init__.pyi +++ b/python/stubs/pyarrow/__init__.pyi @@ -10,6 +10,8 @@ DataType: Any ListType: Any StructType: Any MapType: Any +FixedSizeListType: Any +FixedSizeBinaryType: Any schema: Any map_: Any list_: Any diff --git a/python/stubs/pyarrow_hotfix/__init__.pyi b/python/stubs/pyarrow_hotfix/__init__.pyi new file mode 100644 index 0000000000..e69de29bb2 diff --git a/python/tests/conftest.py b/python/tests/conftest.py index ccd651f7f6..6621bc9afb 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -9,7 +9,7 @@ import pyarrow as pa import pytest -from deltalake import DeltaTable, write_deltalake +from deltalake import DeltaTable, WriterProperties, write_deltalake def wait_till_host_is_available(host: str, timeout_sec: int = 0.5): @@ -58,7 +58,7 @@ def s3_localstack_creds(): "s3", "sync", "--quiet", - "../crates/deltalake-core/tests/data/simple_table", + "../crates/test/tests/data/simple_table", "s3://deltars/simple", "--endpoint-url", endpoint_url, @@ -248,3 +248,21 @@ def sample_table(): "deleted": pa.array([False] * nrows), } ) + + +@pytest.fixture() +def sample_table_with_spaces_numbers(): + nrows = 5 + return pa.table( + { + "1id": pa.array(["1", "2", "3", "4", "5"]), + "price": pa.array(list(range(nrows)), pa.int64()), + "sold items": pa.array(list(range(nrows)), pa.int32()), + "deleted": pa.array([False] * nrows), + } + ) + + +@pytest.fixture() +def writer_properties(): + return WriterProperties(compression="GZIP", compression_level=0) diff --git a/python/tests/data_acceptance/test_reader.py b/python/tests/data_acceptance/test_reader.py index 8d3faff49e..ae77100618 100644 --- a/python/tests/data_acceptance/test_reader.py +++ b/python/tests/data_acceptance/test_reader.py @@ -46,6 +46,7 @@ class ReadCase(NamedTuple): failing_cases = { "multi_partitioned_2": "Waiting for PyArrow 11.0.0 for decimal cast support (#1078)", "multi_partitioned": "Test case handles binary poorly", + "all_primitive_types": "The parquet table written with PySpark incorrectly wrote a timestamp primitive without Timezone information.", } diff --git a/python/tests/pyspark_integration/test_write_to_pyspark.py b/python/tests/pyspark_integration/test_write_to_pyspark.py index 8418f587ca..5cf6490a62 100644 --- a/python/tests/pyspark_integration/test_write_to_pyspark.py +++ b/python/tests/pyspark_integration/test_write_to_pyspark.py @@ -1,4 +1,5 @@ """Tests that deltalake(delta-rs) can write to tables written by PySpark.""" + import pathlib import pyarrow as pa diff --git a/python/tests/pyspark_integration/test_writer_readable.py b/python/tests/pyspark_integration/test_writer_readable.py index ea555074b8..3ade57c6e9 100644 --- a/python/tests/pyspark_integration/test_writer_readable.py +++ b/python/tests/pyspark_integration/test_writer_readable.py @@ -1,4 +1,5 @@ """Test that pyspark can read tables written by deltalake(delta-rs).""" + import pathlib import pyarrow as pa diff --git a/python/tests/test_alter.py b/python/tests/test_alter.py new file mode 100644 index 0000000000..4bc902d330 --- /dev/null +++ b/python/tests/test_alter.py @@ -0,0 +1,115 @@ +import pathlib + +import pyarrow as pa +import pytest + +from deltalake import DeltaTable, write_deltalake +from deltalake.exceptions import DeltaError, DeltaProtocolError + + +def test_add_constraint(tmp_path: pathlib.Path, sample_table: pa.Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint({"check_price": "price >= 0"}) + + last_action = dt.history(1)[0] + assert last_action["operation"] == "ADD CONSTRAINT" + assert dt.version() == 1 + assert dt.metadata().configuration == { + "delta.constraints.check_price": "price >= 0" + } + assert dt.protocol().min_writer_version == 3 + + with pytest.raises(DeltaError): + # Invalid constraint + dt.alter.add_constraint({"check_price": "price < 0"}) + + with pytest.raises(DeltaProtocolError): + data = pa.table( + { + "id": pa.array(["1"]), + "price": pa.array([-1], pa.int64()), + "sold": pa.array(list(range(1)), pa.int32()), + "deleted": pa.array([False] * 1), + } + ) + write_deltalake(tmp_path, data, engine="rust", mode="append") + + +def test_add_multiple_constraints(tmp_path: pathlib.Path, sample_table: pa.Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + with pytest.raises(ValueError): + dt.alter.add_constraint( + {"check_price": "price >= 0", "check_price2": "price >= 0"} + ) + + +def test_add_constraint_roundtrip_metadata( + tmp_path: pathlib.Path, sample_table: pa.Table +): + write_deltalake(tmp_path, sample_table, mode="append", engine="rust") + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint( + {"check_price2": "price >= 0"}, custom_metadata={"userName": "John Doe"} + ) + + assert dt.history(1)[0]["userName"] == "John Doe" + + +def test_drop_constraint(tmp_path: pathlib.Path, sample_table: pa.Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint({"check_price": "price >= 0"}) + assert dt.protocol().min_writer_version == 3 + dt.alter.drop_constraint(name="check_price") + last_action = dt.history(1)[0] + assert last_action["operation"] == "DROP CONSTRAINT" + assert dt.version() == 2 + assert dt.metadata().configuration == {} + assert dt.protocol().min_writer_version == 3 + + +def test_drop_constraint_invalid(tmp_path: pathlib.Path, sample_table: pa.Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint({"check_price": "price >= 0"}) + with pytest.raises(DeltaError): + dt.alter.drop_constraint(name="invalid_constraint_name") + + assert dt.metadata().configuration == { + "delta.constraints.check_price": "price >= 0" + } + assert dt.protocol().min_writer_version == 3 + + +def test_drop_constraint_invalid_ignore(tmp_path: pathlib.Path, sample_table: pa.Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint({"check_price": "price >= 0"}) + dt.alter.drop_constraint(name="invalid_constraint_name", raise_if_not_exists=False) + + +def test_drop_constraint_roundtrip_metadata( + tmp_path: pathlib.Path, sample_table: pa.Table +): + write_deltalake(tmp_path, sample_table, mode="append", engine="rust") + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint({"check_price2": "price >= 0"}) + dt.alter.drop_constraint("check_price2", custom_metadata={"userName": "John Doe"}) + + assert dt.history(1)[0]["userName"] == "John Doe" diff --git a/python/tests/test_benchmark.py b/python/tests/test_benchmark.py index d7299ca684..bfcdd3f310 100644 --- a/python/tests/test_benchmark.py +++ b/python/tests/test_benchmark.py @@ -35,19 +35,6 @@ def test_benchmark_write(benchmark, sample_table, tmp_path, engine): assert dt.to_pyarrow_table().sort_by("i") == sample_table -# TODO: support wrapping PyArrow filesystems -# @pytest.mark.benchmark( -# group="write" -# ) -# def test_benchmark_write_pyarrow(benchmark, sample_table, tmp_path): -# fs = pa_fs.SubTreeFileSystem(str(tmp_path), pa_fs.LocalFileSystem()) - -# benchmark(write_deltalake, str(tmp_path), sample_table, mode="overwrite", filesystem=fs) - -# dt = DeltaTable(str(tmp_path)) -# assert dt.to_pyarrow_table(filesystem=fs).sort_by("i") == sample_table - - @pytest.mark.benchmark(group="read") def test_benchmark_read(benchmark, sample_table, tmp_path): write_deltalake(str(tmp_path), sample_table) diff --git a/python/tests/test_checkpoint.py b/python/tests/test_checkpoint.py index 01c958034e..fa1ae6a8ae 100644 --- a/python/tests/test_checkpoint.py +++ b/python/tests/test_checkpoint.py @@ -27,7 +27,7 @@ def test_checkpoint(tmp_path: pathlib.Path, sample_data: pa.Table): assert checkpoint_path.exists() -def test_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): +def setup_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): tmp_table_path = tmp_path / "path" / "to" / "table" first_log_path = tmp_table_path / "_delta_log" / "00000000000000000000.json" second_log_path = tmp_table_path / "_delta_log" / "00000000000000000001.json" @@ -53,9 +53,33 @@ def test_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): assert first_log_path.exists() assert second_log_path.exists() assert third_log_path.exists() + return delta_table + +def test_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): + delta_table = setup_cleanup_metadata(tmp_path, sample_data) + delta_table.create_checkpoint() delta_table.cleanup_metadata() + tmp_table_path = tmp_path / "path" / "to" / "table" + first_log_path = tmp_table_path / "_delta_log" / "00000000000000000000.json" + second_log_path = tmp_table_path / "_delta_log" / "00000000000000000001.json" + third_log_path = tmp_table_path / "_delta_log" / "00000000000000000002.json" + assert not first_log_path.exists() assert second_log_path.exists() assert third_log_path.exists() + + +def test_cleanup_metadata_no_checkpoint(tmp_path: pathlib.Path, sample_data: pa.Table): + delta_table = setup_cleanup_metadata(tmp_path, sample_data) + delta_table.cleanup_metadata() + + tmp_table_path = tmp_path / "path" / "to" / "table" + first_log_path = tmp_table_path / "_delta_log" / "00000000000000000000.json" + second_log_path = tmp_table_path / "_delta_log" / "00000000000000000001.json" + third_log_path = tmp_table_path / "_delta_log" / "00000000000000000002.json" + + assert first_log_path.exists() + assert second_log_path.exists() + assert third_log_path.exists() diff --git a/python/tests/test_convert_to_delta.py b/python/tests/test_convert_to_delta.py index 29badf3358..164e817fb1 100644 --- a/python/tests/test_convert_to_delta.py +++ b/python/tests/test_convert_to_delta.py @@ -24,6 +24,7 @@ def test_local_convert_to_delta(tmp_path: pathlib.Path, sample_data: pa.Table): name=name, description=description, configuration={"delta.AppendOnly": "True"}, + custom_metadata={"userName": "John Doe"}, ) dt = DeltaTable(tmp_path) @@ -33,6 +34,7 @@ def test_local_convert_to_delta(tmp_path: pathlib.Path, sample_data: pa.Table): assert dt.metadata().name == name assert dt.metadata().description == description assert dt.metadata().configuration == {"delta.AppendOnly": "True"} + assert dt.history()[0]["userName"] == "John Doe" def test_convert_delta_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): diff --git a/python/tests/test_create.py b/python/tests/test_create.py index a618d741a1..3852fc2bab 100644 --- a/python/tests/test_create.py +++ b/python/tests/test_create.py @@ -14,6 +14,7 @@ def test_create_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table name="test_name", description="test_desc", configuration={"delta.appendOnly": "false", "foo": "bar"}, + custom_metadata={"userName": "John Doe"}, ) metadata = dt.metadata() @@ -21,6 +22,7 @@ def test_create_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table assert metadata.name == "test_name" assert metadata.description == "test_desc" assert metadata.configuration == {"delta.appendOnly": "false", "foo": "bar"} + assert dt.history()[0]["userName"] == "John Doe" def test_create_modes(tmp_path: pathlib.Path, sample_data: pa.Table): diff --git a/python/tests/test_delete.py b/python/tests/test_delete.py index f90125b2fb..519af0c935 100644 --- a/python/tests/test_delete.py +++ b/python/tests/test_delete.py @@ -2,6 +2,7 @@ import pyarrow as pa import pyarrow.compute as pc +import pytest from deltalake.table import DeltaTable from deltalake.writer import write_deltalake @@ -10,11 +11,12 @@ def test_delete_no_predicates(existing_table: DeltaTable): old_version = existing_table.version() - existing_table.delete() + existing_table.delete(custom_metadata={"userName": "John Doe"}) last_action = existing_table.history(1)[0] assert last_action["operation"] == "DELETE" assert existing_table.version() == old_version + 1 + assert last_action["userName"] == "John Doe" dataset = existing_table.to_pyarrow_dataset() assert dataset.count_rows() == 0 @@ -56,3 +58,26 @@ def test_delete_some_rows(existing_table: DeltaTable): table = existing_table.to_pyarrow_table() assert table.equals(expected_table) + + +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_delete_large_dtypes( + tmp_path: pathlib.Path, sample_table: pa.table, engine: str +): + write_deltalake(tmp_path, sample_table, large_dtypes=True, engine=engine) # type: ignore + + dt = DeltaTable(tmp_path) + old_version = dt.version() + + existing = dt.to_pyarrow_table() + mask = pc.invert(pc.is_in(existing["id"], pa.array(["1"]))) + expected_table = existing.filter(mask) + + dt.delete(predicate="id = '1'") + + last_action = dt.history(1)[0] + assert last_action["operation"] == "DELETE" + assert dt.version() == old_version + 1 + + table = dt.to_pyarrow_table() + assert table.equals(expected_table) diff --git a/python/tests/test_fs.py b/python/tests/test_fs.py index 92e77e794a..b4973572e9 100644 --- a/python/tests/test_fs.py +++ b/python/tests/test_fs.py @@ -77,11 +77,11 @@ def test_read_simple_table_from_remote(s3_localstack): assert dt.to_pyarrow_table().equals(pa.table({"id": [5, 7, 9]})) expected_files = [ + "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", ] assert dt.files() == expected_files @@ -91,6 +91,9 @@ def test_read_simple_table_from_remote(s3_localstack): @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") +@pytest.mark.skip( + reason="Temporarily disabled until we can resolve https://github.com/delta-io/delta-rs/pull/2120#issuecomment-1912367573" +) def test_roundtrip_s3_env(s3_localstack, sample_data: pa.Table, monkeypatch): table_path = "s3://deltars/roundtrip" @@ -156,20 +159,6 @@ def test_roundtrip_s3_direct(s3_localstack_creds, sample_data: pa.Table): table = dt.to_pyarrow_table() assert table == sample_data - # TODO: Refactor so DeltaTable can be instantiated with a storage backend - # Can provide S3Filesystem from pyarrow - # pa_s3fs = S3FileSystem( - # access_key=s3_localstack_creds["AWS_ACCESS_KEY_ID"], - # secret_key=s3_localstack_creds["AWS_SECRET_ACCESS_KEY"], - # endpoint_override=s3_localstack_creds["AWS_ENDPOINT_URL"], - # scheme="http", - # ) - - # write_deltalake(table_path, sample_data, filesystem=pa_s3fs, mode="overwrite") - # assert dt.version() == 2 - # table = dt.to_pyarrow_table() - # assert table == sample_data - @pytest.mark.azure @pytest.mark.integration diff --git a/python/tests/test_merge.py b/python/tests/test_merge.py index ddc5a34ea1..82776c60fc 100644 --- a/python/tests/test_merge.py +++ b/python/tests/test_merge.py @@ -24,6 +24,7 @@ def test_merge_when_matched_delete_wo_predicate( predicate="t.id = s.id", source_alias="s", target_alias="t", + custom_metadata={"userName": "John Doe"}, ).when_matched_delete().execute() nrows = 4 @@ -39,6 +40,7 @@ def test_merge_when_matched_delete_wo_predicate( last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" + assert last_action["userName"] == "John Doe" assert result == expected @@ -272,7 +274,7 @@ def test_merge_when_not_matched_insert_with_predicate( "sold": "source.sold", "deleted": "False", }, - predicate="source.price < bigint'50'", + predicate="source.price < 50", ).execute() expected = pa.table( @@ -312,7 +314,7 @@ def test_merge_when_not_matched_insert_all_with_predicate( target_alias="target", predicate="target.id = source.id", ).when_not_matched_insert_all( - predicate="source.price < bigint'50'", + predicate="source.price < 50", ).execute() expected = pa.table( @@ -330,6 +332,46 @@ def test_merge_when_not_matched_insert_all_with_predicate( assert result == expected +def test_merge_when_not_matched_insert_all_with_predicate_special_column_names( + tmp_path: pathlib.Path, sample_table_with_spaces_numbers: pa.Table +): + write_deltalake(tmp_path, sample_table_with_spaces_numbers, mode="append") + + dt = DeltaTable(tmp_path) + + source_table = pa.table( + { + "1id": pa.array(["6", "10"]), + "price": pa.array([10, 100], pa.int64()), + "sold items": pa.array([10, 20], pa.int32()), + "deleted": pa.array([None, None], pa.bool_()), + } + ) + + dt.merge( + source=source_table, + source_alias="source", + target_alias="target", + predicate="target.`1id` = source.`1id`", + ).when_not_matched_insert_all( + predicate="source.price < 50", + ).execute() + + expected = pa.table( + { + "1id": pa.array(["1", "2", "3", "4", "5", "6"]), + "price": pa.array([0, 1, 2, 3, 4, 10], pa.int64()), + "sold items": pa.array([0, 1, 2, 3, 4, 10], pa.int32()), + "deleted": pa.array([False, False, False, False, False, None]), + } + ) + result = dt.to_pyarrow_table().sort_by([("1id", "ascending")]) + last_action = dt.history(1)[0] + + assert last_action["operation"] == "MERGE" + assert result == expected + + def test_merge_when_not_matched_by_source_update_wo_predicate( tmp_path: pathlib.Path, sample_table: pa.Table ): @@ -397,7 +439,7 @@ def test_merge_when_not_matched_by_source_update_with_predicate( updates={ "sold": "int'10'", }, - predicate="target.price > bigint'3'", + predicate="target.price > 3", ).execute() expected = pa.table( @@ -436,7 +478,7 @@ def test_merge_when_not_matched_by_source_delete_with_predicate( source_alias="source", target_alias="target", predicate="target.id = source.id", - ).when_not_matched_by_source_delete(predicate="target.price > bigint'3'").execute() + ).when_not_matched_by_source_delete(predicate="target.price > 3").execute() expected = pa.table( { @@ -606,7 +648,7 @@ def test_merge_multiple_when_not_matched_insert_with_predicate( "sold": "source.sold", "deleted": "False", }, - predicate="source.price < bigint'50'", + predicate="source.price < 50", ).when_not_matched_insert( updates={ "id": "source.id", @@ -614,7 +656,7 @@ def test_merge_multiple_when_not_matched_insert_with_predicate( "sold": "source.sold", "deleted": "False", }, - predicate="source.price > bigint'50'", + predicate="source.price > 50", ).execute() expected = pa.table( diff --git a/python/tests/test_optimize.py b/python/tests/test_optimize.py index d999aa66b0..1be0654836 100644 --- a/python/tests/test_optimize.py +++ b/python/tests/test_optimize.py @@ -39,11 +39,12 @@ def test_optimize_run_table( old_data = dt.to_pyarrow_table() old_version = dt.version() - dt.optimize.compact() + dt.optimize.compact(custom_metadata={"userName": "John Doe"}) new_data = dt.to_pyarrow_table() last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" + assert last_action["userName"] == "John Doe" assert dt.version() == old_version + 1 assert old_data == new_data @@ -69,9 +70,12 @@ def test_z_order_optimize( dt = DeltaTable(tmp_path) old_version = dt.version() - dt.optimize.z_order(["date32", "timestamp"]) + dt.optimize.z_order( + ["date32", "timestamp"], custom_metadata={"userName": "John Doe"} + ) last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" + assert last_action["userName"] == "John Doe" assert dt.version() == old_version + 1 assert len(dt.file_uris()) == 1 diff --git a/python/tests/test_repair.py b/python/tests/test_repair.py index d1c631dcf1..1d4a6adfa8 100644 --- a/python/tests/test_repair.py +++ b/python/tests/test_repair.py @@ -23,9 +23,10 @@ def test_repair_wo_dry_run(tmp_path, sample_data): dt = DeltaTable(tmp_path) os.remove(dt.file_uris()[0]) - metrics = dt.repair(dry_run=False) + metrics = dt.repair(dry_run=False, custom_metadata={"userName": "John Doe"}) last_action = dt.history(1)[0] assert len(metrics["files_removed"]) == 1 assert metrics["dry_run"] is False assert last_action["operation"] == "FSCK" + assert last_action["userName"] == "John Doe" diff --git a/python/tests/test_restore.py b/python/tests/test_restore.py index d877b97c40..099b887726 100644 --- a/python/tests/test_restore.py +++ b/python/tests/test_restore.py @@ -24,9 +24,10 @@ def test_restore_with_version( dt = DeltaTable(table_path) old_version = dt.version() - dt.restore(1) + dt.restore(1, custom_metadata={"userName": "John Doe"}) last_action = dt.history(1)[0] assert last_action["operation"] == "RESTORE" + assert last_action["userName"] == "John Doe" assert dt.version() == old_version + 1 diff --git a/python/tests/test_schema.py b/python/tests/test_schema.py index 6a30ca684e..4d70c720dd 100644 --- a/python/tests/test_schema.py +++ b/python/tests/test_schema.py @@ -15,7 +15,7 @@ def test_table_schema(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) schema = dt.schema() assert json.loads(schema.to_json()) == { @@ -37,7 +37,7 @@ def test_table_schema(): def test_table_schema_pyarrow_simple(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) schema = dt.schema().to_pyarrow() field = schema.field(0) @@ -49,7 +49,7 @@ def test_table_schema_pyarrow_simple(): def test_table_schema_pyarrow_020(): - table_path = "../crates/deltalake-core/tests/data/delta-0.2.0" + table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path) schema = dt.schema().to_pyarrow() field = schema.field(0) @@ -235,12 +235,16 @@ def test_delta_schema(): [ pa.field("some_int", pa.uint32(), nullable=True), pa.field("some_string", pa.string(), nullable=False), + pa.field("some_fixed_binary", pa.binary(5), nullable=False), + pa.field("some_decimal", pa.decimal128(10, 2), nullable=False), ] ), pa.schema( [ pa.field("some_int", pa.int32(), nullable=True), pa.field("some_string", pa.string(), nullable=False), + pa.field("some_fixed_binary", pa.binary(), nullable=False), + pa.field("some_decimal", pa.decimal128(10, 2), nullable=False), ] ), False, @@ -293,6 +297,7 @@ def test_delta_schema(): pa.schema( [ ("some_list", pa.list_(pa.string())), + ("some_fixed_list_int", pa.list_(pa.uint32(), 5)), ("some_list_binary", pa.list_(pa.binary())), ("some_string", pa.large_string()), ] @@ -300,6 +305,7 @@ def test_delta_schema(): pa.schema( [ ("some_list", pa.large_list(pa.large_string())), + ("some_fixed_list_int", pa.large_list(pa.int32())), ("some_list_binary", pa.large_list(pa.large_binary())), ("some_string", pa.large_string()), ] @@ -398,10 +404,10 @@ def test_delta_schema(): ("timestamp1", pa.timestamp("us")), ("timestamp2", pa.timestamp("us")), ("timestamp3", pa.timestamp("us")), - ("timestamp4", pa.timestamp("us")), - ("timestamp5", pa.timestamp("us")), - ("timestamp6", pa.timestamp("us")), - ("timestamp7", pa.timestamp("us")), + ("timestamp4", pa.timestamp("us", tz="UTC")), + ("timestamp5", pa.timestamp("us", tz="UTC")), + ("timestamp6", pa.timestamp("us", tz="UTC")), + ("timestamp7", pa.timestamp("us", tz="UTC")), ] ), False, diff --git a/python/tests/test_table_read.py b/python/tests/test_table_read.py index a49374e710..e0ca4467ee 100644 --- a/python/tests/test_table_read.py +++ b/python/tests/test_table_read.py @@ -1,5 +1,5 @@ import os -from datetime import date, datetime +from datetime import date, datetime, timezone from pathlib import Path from threading import Barrier, Thread from types import SimpleNamespace @@ -30,41 +30,57 @@ def test_read_table_with_edge_timestamps(): - table_path = "../crates/deltalake-core/tests/data/table_with_edge_timestamps" + table_path = "../crates/test/tests/data/table_with_edge_timestamps" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset( parquet_read_options=ParquetReadOptions(coerce_int96_timestamp_unit="ms") ) assert dataset.to_table().to_pydict() == { - "BIG_DATE": [datetime(9999, 12, 31, 0, 0, 0), datetime(9999, 12, 30, 0, 0, 0)], - "NORMAL_DATE": [datetime(2022, 1, 1, 0, 0, 0), datetime(2022, 2, 1, 0, 0, 0)], + "BIG_DATE": [ + datetime(9999, 12, 31, 0, 0, 0, tzinfo=timezone.utc), + datetime(9999, 12, 30, 0, 0, 0, tzinfo=timezone.utc), + ], + "NORMAL_DATE": [ + datetime(2022, 1, 1, 0, 0, 0, tzinfo=timezone.utc), + datetime(2022, 2, 1, 0, 0, 0, tzinfo=timezone.utc), + ], "SOME_VALUE": [1, 2], } # Can push down filters to these timestamps. - predicate = ds.field("BIG_DATE") == datetime(9999, 12, 31, 0, 0, 0) + predicate = ds.field("BIG_DATE") == datetime( + 9999, 12, 31, 0, 0, 0, tzinfo=timezone.utc + ) assert len(list(dataset.get_fragments(predicate))) == 1 def test_read_simple_table_to_dict(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"id": [5, 7, 9]} def test_read_simple_table_by_version_to_dict(): - table_path = "../crates/deltalake-core/tests/data/delta-0.2.0" + table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path, version=2) assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"value": [1, 2, 3]} def test_read_simple_table_using_options_to_dict(): - table_path = "../crates/deltalake-core/tests/data/delta-0.2.0" + table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path, version=2, storage_options={}) assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"value": [1, 2, 3]} -def test_load_with_datetime(): - log_dir = "../crates/deltalake-core/tests/data/simple_table/_delta_log" +@pytest.mark.parametrize( + ["date_value", "expected_version"], + [ + ("2020-05-01T00:47:31-07:00", 0), + ("2020-05-02T22:47:31-07:00", 1), + ("2020-05-25T22:47:31-07:00", 4), + ], +) +def test_load_as_version_datetime(date_value: str, expected_version): + log_dir = "../crates/test/tests/data/simple_table/_delta_log" log_mtime_pair = [ ("00000000000000000000.json", 1588398451.0), ("00000000000000000001.json", 1588484851.0), @@ -76,18 +92,17 @@ def test_load_with_datetime(): file_path = os.path.join(log_dir, file_name) os.utime(file_path, (dt_epoch, dt_epoch)) - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" + dt = DeltaTable(table_path) + dt.load_as_version(date_value) + assert dt.version() == expected_version dt = DeltaTable(table_path) - dt.load_with_datetime("2020-05-01T00:47:31-07:00") - assert dt.version() == 0 - dt.load_with_datetime("2020-05-02T22:47:31-07:00") - assert dt.version() == 1 - dt.load_with_datetime("2020-05-25T22:47:31-07:00") - assert dt.version() == 4 + dt.load_as_version(datetime.fromisoformat(date_value)) + assert dt.version() == expected_version -def test_load_with_datetime_bad_format(): - table_path = "../crates/deltalake-core/tests/data/simple_table" +def test_load_as_version_datetime_bad_format(): + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) for bad_format in [ @@ -96,11 +111,11 @@ def test_load_with_datetime_bad_format(): "2020-05-01T00:47:31+08", ]: with pytest.raises(Exception, match="Failed to parse datetime string:"): - dt.load_with_datetime(bad_format) + dt.load_as_version(bad_format) def test_read_simple_table_update_incremental(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path, version=0) assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"id": [0, 1, 2, 3, 4]} dt.update_incremental() @@ -108,7 +123,7 @@ def test_read_simple_table_update_incremental(): def test_read_simple_table_file_sizes_failure(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) add_actions = dt.get_add_actions().to_pydict() @@ -125,7 +140,7 @@ def test_read_simple_table_file_sizes_failure(): def test_read_partitioned_table_to_dict(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { "value": ["1", "2", "3", "6", "7", "5", "4"], @@ -137,7 +152,7 @@ def test_read_partitioned_table_to_dict(): def test_read_partitioned_table_with_partitions_filters_to_dict(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) partitions = [("year", "=", "2021")] expected = { @@ -151,7 +166,7 @@ def test_read_partitioned_table_with_partitions_filters_to_dict(): def test_read_empty_delta_table_after_delete(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8-empty" + table_path = "../crates/test/tests/data/delta-0.8-empty" dt = DeltaTable(table_path) expected = {"column": []} @@ -159,7 +174,7 @@ def test_read_empty_delta_table_after_delete(): def test_read_table_with_column_subset(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { "value": ["1", "2", "3", "6", "7", "5", "4"], @@ -172,7 +187,7 @@ def test_read_table_with_column_subset(): def test_read_table_as_category(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) assert dt.schema().to_pyarrow().field("value").type == pa.string() @@ -186,7 +201,7 @@ def test_read_table_as_category(): def test_read_table_with_filter(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { "value": ["6", "7", "5"], @@ -203,7 +218,7 @@ def test_read_table_with_filter(): def test_read_table_with_stats(): - table_path = "../crates/deltalake-core/tests/data/COVID-19_NYT" + table_path = "../crates/test/tests/data/COVID-19_NYT" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset() @@ -229,7 +244,7 @@ def test_read_table_with_stats(): def test_read_special_partition(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-special-partition" + table_path = "../crates/test/tests/data/delta-0.8.0-special-partition" dt = DeltaTable(table_path) file1 = ( @@ -251,7 +266,7 @@ def test_read_special_partition(): def test_read_partitioned_table_metadata(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) metadata = dt.metadata() assert metadata.id == "fe5a3c11-30d4-4dd7-b115-a1c121e66a4e" @@ -263,7 +278,7 @@ def test_read_partitioned_table_metadata(): def test_read_partitioned_table_protocol(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) protocol = dt.protocol() assert protocol.min_reader_version == 1 @@ -271,7 +286,7 @@ def test_read_partitioned_table_protocol(): def test_read_table_with_cdc(): - table_path = "../crates/deltalake-core/tests/data/simple_table_with_cdc" + table_path = "../crates/test/tests/data/simple_table_with_cdc" dt = DeltaTable(table_path) assert dt.to_pyarrow_table().to_pydict() == { "id": [0], @@ -280,7 +295,7 @@ def test_read_table_with_cdc(): def test_history_partitioned_table_metadata(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) history = dt.history() commit_info = history[0] @@ -305,7 +320,7 @@ def test_history_partitioned_table_metadata(): @pytest.mark.parametrize("flatten", [True, False]) def test_add_actions_table(flatten: bool): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) actions_df = dt.get_add_actions(flatten) # RecordBatch doesn't have a sort_by method yet @@ -349,10 +364,10 @@ def assert_correct_files(dt: DeltaTable, partition_filters, expected_paths): def test_get_files_partitioned_table(): - table_path = "../crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) table_path = ( - Path.cwd().parent / "crates/deltalake-core/tests/data/delta-0.8.0-partitioned" + Path.cwd().parent / "crates/test/tests/data/delta-0.8.0-partitioned" ).as_posix() partition_filters = [("day", "=", "3")] @@ -425,14 +440,14 @@ def test_get_files_partitioned_table(): @pytest.mark.pandas def test_delta_table_to_pandas(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) assert dt.to_pandas().equals(pd.DataFrame({"id": [5, 7, 9]})) @pytest.mark.pandas def test_delta_table_with_filesystem(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) filesystem = SubTreeFileSystem(table_path, LocalFileSystem()) assert dt.to_pandas(filesystem=filesystem).equals(pd.DataFrame({"id": [5, 7, 9]})) @@ -440,7 +455,7 @@ def test_delta_table_with_filesystem(): @pytest.mark.pandas def test_delta_table_with_filters(): - table_path = "../crates/deltalake-core/tests/data/COVID-19_NYT" + table_path = "../crates/test/tests/data/COVID-19_NYT" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset() @@ -482,9 +497,9 @@ def test_delta_table_with_filters(): def test_writer_fails_on_protocol(): - table_path = "../crates/deltalake-core/tests/data/simple_table" + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) - dt.protocol = Mock(return_value=ProtocolVersions(2, 1)) + dt.protocol = Mock(return_value=ProtocolVersions(2, 1, None, None)) with pytest.raises(DeltaProtocolError): dt.to_pyarrow_dataset() with pytest.raises(DeltaProtocolError): @@ -540,16 +555,16 @@ def join(self, timeout=None): @pytest.mark.timeout(timeout=5, method="thread") def test_read_multiple_tables_from_s3(s3_localstack): """Should be able to create multiple cloud storage based DeltaTable instances - without blocking on async crates/deltalake-core function calls. + without blocking on async crates/test function calls. """ for path in ["s3://deltars/simple", "s3://deltars/simple"]: t = DeltaTable(path) assert t.files() == [ + "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", ] @@ -565,11 +580,11 @@ def read_table(): b.wait() t = DeltaTable("s3://deltars/simple") assert t.files() == [ + "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", ] threads = [ExcPassThroughThread(target=read_table) for _ in range(thread_count)] @@ -725,3 +740,9 @@ def test_encode_partition_value(input_value: Any, expected: str) -> None: assert [encode_partition_value(val) for val in input_value] == expected else: assert encode_partition_value(input_value) == expected + + +def test_read_table_last_checkpoint_not_updated(): + dt = DeltaTable("../crates/test/tests/data/table_failed_last_checkpoint_update") + + assert dt.version() == 3 diff --git a/python/tests/test_update.py b/python/tests/test_update.py index 2e3fc82fdd..fcc17cf027 100644 --- a/python/tests/test_update.py +++ b/python/tests/test_update.py @@ -38,7 +38,43 @@ def test_update_with_predicate(tmp_path: pathlib.Path, sample_table: pa.Table): } ) - dt.update(updates={"deleted": "True"}, predicate="price > 3") + dt.update( + updates={"deleted": "True"}, + predicate="price > 3", + custom_metadata={"userName": "John Doe"}, + ) + + result = dt.to_pyarrow_table() + last_action = dt.history(1)[0] + + assert last_action["operation"] == "UPDATE" + assert last_action["userName"] == "John Doe" + assert result == expected + + +def test_update_with_predicate_large_dtypes( + tmp_path: pathlib.Path, sample_table: pa.Table +): + write_deltalake(tmp_path, sample_table, mode="append", large_dtypes=True) + + dt = DeltaTable(tmp_path) + + nrows = 5 + expected = pa.table( + { + "id": pa.array(["1", "2", "3", "4", "5"]), + "price": pa.array(list(range(nrows)), pa.int64()), + "sold": pa.array(list(range(nrows)), pa.int64()), + "price_float": pa.array(list(range(nrows)), pa.float64()), + "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), + "deleted": pa.array([True, False, False, False, False]), + } + ) + + dt.update( + updates={"deleted": "True"}, + predicate="id = '1'", + ) result = dt.to_pyarrow_table() last_action = dt.history(1)[0] diff --git a/python/tests/test_vacuum.py b/python/tests/test_vacuum.py index 3f8c010bc5..44c2195e17 100644 --- a/python/tests/test_vacuum.py +++ b/python/tests/test_vacuum.py @@ -8,7 +8,7 @@ def test_vacuum_dry_run_simple_table(): - table_path = "../crates/deltalake-core/tests/data/delta-0.2.0" + table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path) retention_periods = 169 tombstones = dt.vacuum(retention_periods) @@ -72,7 +72,12 @@ def test_vacuum_transaction_log(tmp_path: pathlib.Path, sample_data: pa.Table): dt = DeltaTable(tmp_path) - dt.vacuum(retention_hours=0, dry_run=False, enforce_retention_duration=False) + dt.vacuum( + retention_hours=0, + dry_run=False, + enforce_retention_duration=False, + custom_metadata={"userName": "John Doe"}, + ) dt = DeltaTable(tmp_path) @@ -87,6 +92,9 @@ def test_vacuum_transaction_log(tmp_path: pathlib.Path, sample_data: pa.Table): assert history[0]["operation"] == "VACUUM END" assert history[1]["operation"] == "VACUUM START" + assert history[0]["userName"] == "John Doe" + assert history[1]["userName"] == "John Doe" + assert history[0]["operationParameters"]["status"] == "COMPLETED" assert history[1]["operationParameters"] == expected_start_parameters diff --git a/python/tests/test_version.py b/python/tests/test_version.py index 8cf22d8045..df1442a66e 100644 --- a/python/tests/test_version.py +++ b/python/tests/test_version.py @@ -1,6 +1,6 @@ from deltalake import rust_core_version -def test_read_simple_table_to_dict() -> None: +def test_version() -> None: v = rust_core_version() assert len(v.split(".")) == 3 diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 49177782ff..dfd124a73d 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -17,7 +17,12 @@ from pyarrow.lib import RecordBatchReader from deltalake import DeltaTable, Schema, write_deltalake -from deltalake.exceptions import CommitFailedError, DeltaError, DeltaProtocolError +from deltalake.exceptions import ( + CommitFailedError, + DeltaError, + DeltaProtocolError, + SchemaMismatchError, +) from deltalake.table import ProtocolVersions from deltalake.writer import try_get_table_and_table_uri @@ -124,11 +129,17 @@ def test_enforce_schema(existing_table: DeltaTable, mode: str): def test_enforce_schema_rust_writer(existing_table: DeltaTable, mode: str): bad_data = pa.table({"x": pa.array([1, 2, 3])}) - with pytest.raises(DeltaError): + with pytest.raises( + SchemaMismatchError, + match=".*Cannot cast schema, number of fields does not match.*", + ): write_deltalake(existing_table, bad_data, mode=mode, engine="rust") table_uri = existing_table._table.table_uri() - with pytest.raises(DeltaError): + with pytest.raises( + SchemaMismatchError, + match=".*Cannot cast schema, number of fields does not match.*", + ): write_deltalake(table_uri, bad_data, mode=mode, engine="rust") @@ -136,48 +147,154 @@ def test_update_schema(existing_table: DeltaTable): new_data = pa.table({"x": pa.array([1, 2, 3])}) with pytest.raises(ValueError): - write_deltalake(existing_table, new_data, mode="append", overwrite_schema=True) + write_deltalake( + existing_table, new_data, mode="append", schema_mode="overwrite" + ) - write_deltalake(existing_table, new_data, mode="overwrite", overwrite_schema=True) + write_deltalake(existing_table, new_data, mode="overwrite", schema_mode="overwrite") read_data = existing_table.to_pyarrow_table() assert new_data == read_data assert existing_table.schema().to_pyarrow() == new_data.schema -def test_update_schema_rust_writer(existing_table: DeltaTable): - new_data = pa.table({"x": pa.array([1, 2, 3])}) +def test_merge_schema(existing_table: DeltaTable): + print(existing_table._table.table_uri()) + old_table_data = existing_table.to_pyarrow_table() + new_data = pa.table( + { + "new_x": pa.array([1, 2, 3], pa.int32()), + "new_y": pa.array([1, 2, 3], pa.int32()), + } + ) + + write_deltalake( + existing_table, new_data, mode="append", schema_mode="merge", engine="rust" + ) + # adjust schema of old_table_data and new_data to match each other + + for i in range(old_table_data.num_columns): + col = old_table_data.schema.field(i) + new_data = new_data.add_column(i, col, pa.nulls(new_data.num_rows, col.type)) + + old_table_data = old_table_data.append_column( + pa.field("new_x", pa.int32()), pa.nulls(old_table_data.num_rows, pa.int32()) + ) + old_table_data = old_table_data.append_column( + pa.field("new_y", pa.int32()), pa.nulls(old_table_data.num_rows, pa.int32()) + ) + + # define sort order + read_data = existing_table.to_pyarrow_table().sort_by( + [("utf8", "ascending"), ("new_x", "ascending")] + ) + print(repr(read_data.to_pylist())) + concated = pa.concat_tables([old_table_data, new_data]) + print(repr(concated.to_pylist())) + assert read_data == concated + + write_deltalake(existing_table, new_data, mode="overwrite", schema_mode="overwrite") + + assert existing_table.schema().to_pyarrow() == new_data.schema + + +def test_overwrite_schema(existing_table: DeltaTable): + new_data_invalid = pa.table( + { + "utf8": pa.array([1235, 546, 5645]), + "new_x": pa.array([1, 2, 3], pa.int32()), + "new_y": pa.array([1, 2, 3], pa.int32()), + } + ) with pytest.raises(DeltaError): write_deltalake( existing_table, - new_data, + new_data_invalid, mode="append", - overwrite_schema=True, + schema_mode="overwrite", engine="rust", ) + + new_data = pa.table( + { + "utf8": pa.array(["bla", "bli", "blubb"]), + "new_x": pa.array([1, 2, 3], pa.int32()), + "new_y": pa.array([1, 2, 3], pa.int32()), + } + ) with pytest.raises(DeltaError): write_deltalake( existing_table, new_data, - mode="overwrite", - overwrite_schema=False, + mode="append", + schema_mode="overwrite", + engine="rust", + ) + + write_deltalake(existing_table, new_data, mode="overwrite", schema_mode="overwrite") + + assert existing_table.schema().to_pyarrow() == new_data.schema + + +def test_update_schema_rust_writer_append(existing_table: DeltaTable): + with pytest.raises( + SchemaMismatchError, match="Cannot cast schema, number of fields does not match" + ): + # It's illegal to do schema drift without correct schema_mode + write_deltalake( + existing_table, + pa.table({"x4": pa.array([1, 2, 3])}), + mode="append", + schema_mode=None, engine="rust", ) with pytest.raises(DeltaError): + write_deltalake( # schema_mode overwrite is illegal with append + existing_table, + pa.table({"x1": pa.array([1, 2, 3])}), + mode="append", + schema_mode="overwrite", + engine="rust", + ) + with pytest.raises( + SchemaMismatchError, + match="Schema error: Fail to merge schema field 'utf8' because the from data_type = Int64 does not equal Utf8", + ): write_deltalake( existing_table, - new_data, + pa.table({"utf8": pa.array([1, 2, 3])}), mode="append", - overwrite_schema=False, + schema_mode="merge", engine="rust", ) - # TODO(ion): Remove this once we add schema overwrite support + write_deltalake( + existing_table, + pa.table({"x2": pa.array([1, 2, 3])}), + mode="append", + schema_mode="merge", + engine="rust", + ) + + +def test_update_schema_rust_writer_invalid(existing_table: DeltaTable): + new_data = pa.table({"x5": pa.array([1, 2, 3])}) + with pytest.raises( + SchemaMismatchError, match="Cannot cast schema, number of fields does not match" + ): + write_deltalake( + existing_table, + new_data, + mode="overwrite", + schema_mode=None, + engine="rust", + ) + write_deltalake( existing_table, new_data, mode="overwrite", - overwrite_schema=True, + schema_mode="overwrite", engine="rust", ) @@ -205,6 +322,30 @@ def test_local_path( assert table == sample_data +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_local_path_with_unsafe_rename( + tmp_path: pathlib.Path, + sample_data: pa.Table, + monkeypatch, + engine: Literal["pyarrow", "rust"], +): + monkeypatch.chdir(tmp_path) # Make tmp_path the working directory + (tmp_path / "path/to/table").mkdir(parents=True) + + local_path = "./path/to/table" + storage_opts = { + "allow_unsafe_rename": "true", + } + write_deltalake( + local_path, sample_data, storage_options=storage_opts, engine=engine + ) + delta_table = DeltaTable(local_path, storage_options=storage_opts) + assert delta_table.schema().to_pyarrow() == sample_data.schema + + table = delta_table.to_pyarrow_table() + assert table == sample_data + + @pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table, engine): write_deltalake( @@ -237,7 +378,7 @@ def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table, engin "float32", "float64", "bool", - "binary", + # "binary", "date32", "timestamp", ], @@ -304,7 +445,7 @@ def test_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table, engine): assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data if engine == "pyarrow": - with pytest.raises(AssertionError): + with pytest.raises(FileExistsError): write_deltalake(tmp_path, sample_data, mode="error") elif engine == "rust": with pytest.raises(DeltaError): @@ -364,7 +505,7 @@ def test_fails_wrong_partitioning( existing_table: DeltaTable, sample_data: pa.Table, engine ): if engine == "pyarrow": - with pytest.raises(AssertionError): + with pytest.raises(ValueError): write_deltalake( existing_table, sample_data, @@ -587,7 +728,7 @@ def test_writer_fails_on_protocol( sample_data: pa.Table, engine: Literal["pyarrow", "rust"], ): - existing_table.protocol = Mock(return_value=ProtocolVersions(1, 3)) + existing_table.protocol = Mock(return_value=ProtocolVersions(1, 3, None, None)) with pytest.raises(DeltaProtocolError): write_deltalake(existing_table, sample_data, mode="overwrite", engine=engine) @@ -661,35 +802,58 @@ def test_writer_with_options(tmp_path: pathlib.Path): def test_try_get_table_and_table_uri(tmp_path: pathlib.Path): + def _normalize_path(t): # who does not love Windows? ;) + return t[0], t[1].replace("\\", "/") if t[1] else t[1] + data = pa.table({"vals": pa.array(["1", "2", "3"])}) table_or_uri = tmp_path / "delta_table" write_deltalake(table_or_uri, data) delta_table = DeltaTable(table_or_uri) # table_or_uri as DeltaTable - assert try_get_table_and_table_uri(delta_table, None) == ( - delta_table, - str(tmp_path / "delta_table") + "/", + assert _normalize_path( + try_get_table_and_table_uri(delta_table, None) + ) == _normalize_path( + ( + delta_table, + str(tmp_path / "delta_table") + "/", + ) ) # table_or_uri as str - assert try_get_table_and_table_uri(str(tmp_path / "delta_table"), None) == ( - delta_table, - str(tmp_path / "delta_table"), + assert _normalize_path( + try_get_table_and_table_uri(str(tmp_path / "delta_table"), None) + ) == _normalize_path( + ( + delta_table, + str(tmp_path / "delta_table"), + ) ) - assert try_get_table_and_table_uri(str(tmp_path / "str"), None) == ( - None, - str(tmp_path / "str"), + assert _normalize_path( + try_get_table_and_table_uri(str(tmp_path / "str"), None) + ) == _normalize_path( + ( + None, + str(tmp_path / "str"), + ) ) # table_or_uri as Path - assert try_get_table_and_table_uri(tmp_path / "delta_table", None) == ( - delta_table, - str(tmp_path / "delta_table"), + assert _normalize_path( + try_get_table_and_table_uri(tmp_path / "delta_table", None) + ) == _normalize_path( + ( + delta_table, + str(tmp_path / "delta_table"), + ) ) - assert try_get_table_and_table_uri(tmp_path / "Path", None) == ( - None, - str(tmp_path / "Path"), + assert _normalize_path( + try_get_table_and_table_uri(tmp_path / "Path", None) + ) == _normalize_path( + ( + None, + str(tmp_path / "Path"), + ) ) # table_or_uri with invalid parameter type @@ -854,6 +1018,7 @@ def test_partition_overwrite_unfiltered_data_fails( ) +@pytest.mark.parametrize("large_dtypes", [True, False]) @pytest.mark.parametrize( "value_1,value_2,value_type,filter_string", [ @@ -868,7 +1033,10 @@ def test_replace_where_overwrite( value_2: Any, value_type: pa.DataType, filter_string: str, + large_dtypes: bool, ): + table_path = tmp_path + sample_data = pa.table( { "p1": pa.array(["1", "1", "2", "2"], pa.string()), @@ -876,9 +1044,11 @@ def test_replace_where_overwrite( "val": pa.array([1, 1, 1, 1], pa.int64()), } ) - write_deltalake(tmp_path, sample_data, mode="overwrite", partition_by=["p1", "p2"]) + write_deltalake( + table_path, sample_data, mode="overwrite", large_dtypes=large_dtypes + ) - delta_table = DeltaTable(tmp_path) + delta_table = DeltaTable(table_path) assert ( delta_table.to_pyarrow_table().sort_by( [("p1", "ascending"), ("p2", "ascending")] @@ -890,36 +1060,102 @@ def test_replace_where_overwrite( { "p1": pa.array(["1", "1"], pa.string()), "p2": pa.array([value_2, value_1], value_type), - "val": pa.array([2, 2], pa.int64()), + "val": pa.array([2, 3], pa.int64()), } ) expected_data = pa.table( { "p1": pa.array(["1", "1", "2", "2"], pa.string()), "p2": pa.array([value_1, value_2, value_1, value_2], value_type), - "val": pa.array([2, 2, 1, 1], pa.int64()), + "val": pa.array([3, 2, 1, 1], pa.int64()), } ) - with pytest.raises( - DeltaError, - match="Generic DeltaTable error: Overwriting data based on predicate is not yet implemented", - ): - write_deltalake( - tmp_path, - sample_data, - mode="overwrite", - predicate="`p1` = 1", - engine="rust", + write_deltalake( + table_path, + sample_data, + mode="overwrite", + predicate="p1 = '1'", + engine="rust", + large_dtypes=large_dtypes, + ) + + delta_table.update_incremental() + assert ( + delta_table.to_pyarrow_table().sort_by( + [("p1", "ascending"), ("p2", "ascending")] ) + == expected_data + ) - delta_table.update_incremental() - assert ( - delta_table.to_pyarrow_table().sort_by( - [("p1", "ascending"), ("p2", "ascending")] - ) - == expected_data + +@pytest.mark.parametrize( + "value_1,value_2,value_type,filter_string", + [ + (1, 2, pa.int64(), "1"), + (False, True, pa.bool_(), "false"), + (date(2022, 1, 1), date(2022, 1, 2), pa.date32(), "2022-01-01"), + ], +) +def test_replace_where_overwrite_partitioned( + tmp_path: pathlib.Path, + value_1: Any, + value_2: Any, + value_type: pa.DataType, + filter_string: str, +): + table_path = tmp_path + + sample_data = pa.table( + { + "p1": pa.array(["1", "1", "2", "2"], pa.string()), + "p2": pa.array([value_1, value_2, value_1, value_2], value_type), + "val": pa.array([1, 1, 1, 1], pa.int64()), + } + ) + write_deltalake( + table_path, sample_data, mode="overwrite", partition_by=["p1", "p2"] + ) + + delta_table = DeltaTable(table_path) + assert ( + delta_table.to_pyarrow_table().sort_by( + [("p1", "ascending"), ("p2", "ascending")] ) + == sample_data + ) + + replace_data = pa.table( + { + "p1": pa.array(["1", "1"], pa.string()), + "p2": pa.array([value_2, value_1], value_type), + "val": pa.array([2, 3], pa.int64()), + } + ) + expected_data = pa.table( + { + "p1": pa.array(["1", "1", "2", "2"], pa.string()), + "p2": pa.array([value_1, value_2, value_1, value_2], value_type), + "val": pa.array([3, 2, 1, 1], pa.int64()), + } + ) + + write_deltalake( + table_path, + replace_data, + mode="overwrite", + partition_by=["p1", "p2"], + predicate="p1 = '1'", + engine="rust", + ) + + delta_table.update_incremental() + assert ( + delta_table.to_pyarrow_table().sort_by( + [("p1", "ascending"), ("p2", "ascending")] + ) + == expected_data + ) def test_partition_overwrite_with_new_partition( @@ -1184,3 +1420,65 @@ def test_with_deltalake_schema(tmp_path: pathlib.Path, sample_data: pa.Table): ) delta_table = DeltaTable(tmp_path) assert delta_table.schema().to_pyarrow() == sample_data.schema + + +def test_write_stats_empty_rowgroups(tmp_path: pathlib.Path): + # https://github.com/delta-io/delta-rs/issues/2169 + data = pa.table( + { + "data": pa.array(["B"] * 1024 * 33), + } + ) + write_deltalake( + tmp_path, + data, + max_rows_per_file=1024 * 32, + max_rows_per_group=1024 * 16, + min_rows_per_group=8 * 1024, + mode="overwrite", + ) + dt = DeltaTable(tmp_path) + assert ( + dt.to_pyarrow_dataset().to_table(filter=(pc.field("data") == "B")).shape[0] + == 33792 + ) + + +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_schema_cols_diff_order(tmp_path: pathlib.Path, engine): + data = pa.table( + { + "foo": pa.array(["B"] * 10), + "bar": pa.array([1] * 10), + "baz": pa.array([2.0] * 10), + } + ) + write_deltalake(tmp_path, data, mode="append", engine=engine) + + data = pa.table( + { + "baz": pa.array([2.0] * 10), + "bar": pa.array([1] * 10), + "foo": pa.array(["B"] * 10), + } + ) + write_deltalake(tmp_path, data, mode="append", engine=engine) + dt = DeltaTable(tmp_path) + assert dt.version() == 1 + + expected = pa.table( + { + "baz": pa.array([2.0] * 20), + "bar": pa.array([1] * 20), + "foo": pa.array(["B"] * 20), + } + ) + + assert dt.to_pyarrow_table(columns=["baz", "bar", "foo"]) == expected + + +def test_empty(existing_table: DeltaTable): + schema = existing_table.schema().to_pyarrow() + empty_table = pa.Table.from_pylist([], schema=schema) + with pytest.raises(DeltaError, match="No data source supplied to write command"): + write_deltalake(existing_table, empty_table, mode="append", engine="rust") diff --git a/python/tests/test_writerproperties.py b/python/tests/test_writerproperties.py new file mode 100644 index 0000000000..63f12879e6 --- /dev/null +++ b/python/tests/test_writerproperties.py @@ -0,0 +1,88 @@ +import pathlib + +import pyarrow as pa +import pyarrow.parquet as pq +import pytest + +from deltalake import DeltaTable, WriterProperties, write_deltalake + + +def test_writer_properties_all_filled(): + wp = WriterProperties( + data_page_size_limit=100, + dictionary_page_size_limit=200, + data_page_row_count_limit=300, + write_batch_size=400, + max_row_group_size=500, + compression="SNAPPY", + ) + + expected = { + "data_page_size_limit": "100", + "dictionary_page_size_limit": "200", + "data_page_row_count_limit": "300", + "write_batch_size": "400", + "max_row_group_size": "500", + "compression": "SNAPPY", + } + + assert wp._to_dict() == expected + + +def test_writer_properties_lower_case_compression(): + wp = WriterProperties(compression="snappy") # type: ignore + + expected = { + "data_page_size_limit": None, + "dictionary_page_size_limit": None, + "data_page_row_count_limit": None, + "write_batch_size": None, + "max_row_group_size": None, + "compression": "SNAPPY", + } + + assert wp._to_dict() == expected + + +@pytest.mark.parametrize( + "compression,expected", + [("GZIP", "GZIP(6)"), ("BROTLI", "BROTLI(1)"), ("ZSTD", "ZSTD(1)")], +) +def test_writer_properties_missing_compression_level(compression, expected): + wp = WriterProperties(compression=compression) + + assert wp.compression == expected + + +@pytest.mark.parametrize( + "compression,wrong_level", + [ + ("GZIP", -1), + ("GZIP", 11), + ("BROTLI", -1), + ("BROTLI", 12), + ("ZSTD", 0), + ("ZSTD", 23), + ], +) +def test_writer_properties_incorrect_level_range(compression, wrong_level): + with pytest.raises(ValueError): + WriterProperties(compression=compression, compression_level=wrong_level) + + +def test_writer_properties_no_compression(): + with pytest.raises(ValueError): + WriterProperties(compression_level=10) + + +def test_write_with_writerproperties( + tmp_path: pathlib.Path, sample_table: pa.Table, writer_properties: WriterProperties +): + write_deltalake( + tmp_path, sample_table, engine="rust", writer_properties=writer_properties + ) + + parquet_path = DeltaTable(tmp_path).file_uris()[0] + metadata = pq.read_metadata(parquet_path) + + assert metadata.to_dict()["row_groups"][0]["columns"][0]["compression"] == "GZIP"