diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 480bc5c521..beb7bbb591 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,4 +1,7 @@ -* @houqp @xianwill @wjones127 @fvaleye @roeap @rtyler @mosyp +crates/ @wjones127 @roeap @rtyler +delta-inspect/ @wjones127 @rtyler proofs/ @houqp -python/ @wjones127 @fvaleye @rtyler @roeap @houqp +python/ @wjones127 @fvaleye @roeap tlaplus/ @houqp +.github/ @wjones127 @rtyler +docs/ @MrPowers diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index ba2915cdc8..80dec2eaef 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -102,7 +102,7 @@ jobs: AWS_ACCESS_KEY_ID: deltalake AWS_SECRET_ACCESS_KEY: weloverust AWS_ENDPOINT_URL: http://localhost:4566 - AWS_STORAGE_ALLOW_HTTP: "1" + AWS_ALLOW_HTTP: "1" AZURE_USE_EMULATOR: "1" AZURE_STORAGE_ALLOW_HTTP: "1" AZURITE_BLOB_STORAGE_URL: "http://localhost:10000" @@ -164,5 +164,5 @@ jobs: - uses: Swatinem/rust-cache@v2 - name: Run tests - working-directory: rust + 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 new file mode 100644 index 0000000000..1e828c5144 --- /dev/null +++ b/.github/workflows/docs.yml @@ -0,0 +1,64 @@ +name: Build documentation + +on: + pull_request: + paths: + - python/** + - docs/** + - mkdocs.yml + - .github/workflows/docs.yml +jobs: + markdown-link-check: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - uses: gaurav-nelson/github-action-markdown-link-check@v1 + with: + config-file: docs/mlc-config.json + folder-path: docs + + lint: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - uses: psf/black@stable + with: + src: docs/src/python + + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + + - name: Install Rust + uses: actions-rs/toolchain@v1 + with: + toolchain: stable + override: true + components: rustfmt, clippy + + - uses: Swatinem/rust-cache@v2 + + - name: Set up Python + uses: actions/setup-python@v3 + with: + python-version: '3.10' + + - name: Build and install deltalake + run: | + cd python + pip install virtualenv + virtualenv venv + source venv/bin/activate + make develop + cd .. + + - name: Install dependencies + run: | + source python/venv/bin/activate + pip install -r docs/requirements.txt + + - name: Build documentation + run: | + source python/venv/bin/activate + mkdocs build \ No newline at end of file diff --git a/.github/workflows/python_release.yml b/.github/workflows/python_release.yml index 85af4ea95a..6793d129a0 100644 --- a/.github/workflows/python_release.yml +++ b/.github/workflows/python_release.yml @@ -114,8 +114,6 @@ jobs: target: aarch64-unknown-linux-gnu command: publish args: --skip-existing -m python/Cargo.toml --no-sdist ${{ env.FEATURES_FLAG }} - # for openssl build - before-script-linux: yum install -y perl-IPC-Cmd release-docs: needs: diff --git a/.gitignore b/.gitignore index 5fe8f6cf0a..ca0576b47c 100644 --- a/.gitignore +++ b/.gitignore @@ -27,3 +27,6 @@ Cargo.lock !/delta-inspect/Cargo.lock !/proofs/Cargo.lock +justfile +site +__pycache__ \ No newline at end of file diff --git a/CHANGELOG.md b/CHANGELOG.md index 6f62ca2dcd..922a49f47e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,86 @@ # Changelog +## [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) + +**Implemented enhancements:** + +- Expose Optimize option min\_commit\_interval in Python [\#1640](https://github.com/delta-io/delta-rs/issues/1640) +- Expose create\_checkpoint\_for [\#1513](https://github.com/delta-io/delta-rs/issues/1513) +- integration tests regularly fail for HDFS [\#1428](https://github.com/delta-io/delta-rs/issues/1428) +- Add Support for Microsoft OneLake [\#1418](https://github.com/delta-io/delta-rs/issues/1418) +- add support for atomic rename in R2 [\#1356](https://github.com/delta-io/delta-rs/issues/1356) + +**Fixed bugs:** + +- Writing with large arrow types \(e.g. large\_utf8\), writes wrong partition encoding [\#1669](https://github.com/delta-io/delta-rs/issues/1669) +- \[python\] Different stringification of partition values in reader and writer [\#1653](https://github.com/delta-io/delta-rs/issues/1653) +- Unable to interface with data written from Spark Databricks [\#1651](https://github.com/delta-io/delta-rs/issues/1651) +- `get_last_checkpoint` does some unnecessary listing [\#1643](https://github.com/delta-io/delta-rs/issues/1643) +- `PartitionWriter`'s `buffer_len` doesn't include incomplete row groups [\#1637](https://github.com/delta-io/delta-rs/issues/1637) +- Slack community invite link has expired [\#1636](https://github.com/delta-io/delta-rs/issues/1636) +- delta-rs does not appear to support tables with liquid clustering [\#1626](https://github.com/delta-io/delta-rs/issues/1626) +- Internal Parquet panic when using a Map type. [\#1619](https://github.com/delta-io/delta-rs/issues/1619) +- partition\_by with "$" on local filesystem [\#1591](https://github.com/delta-io/delta-rs/issues/1591) +- ProtocolChanged error when perfoming append write [\#1585](https://github.com/delta-io/delta-rs/issues/1585) +- Unable to `cargo update` using git tag or rev on Rust 1.70 [\#1580](https://github.com/delta-io/delta-rs/issues/1580) +- NoMetadata error when reading detlatable [\#1562](https://github.com/delta-io/delta-rs/issues/1562) +- Cannot read delta table: `Delta protocol violation` [\#1557](https://github.com/delta-io/delta-rs/issues/1557) +- Update the CODEOWNERS to capture the current reviewers and contributors [\#1553](https://github.com/delta-io/delta-rs/issues/1553) +- \[Python\] Incorrect file URIs when partition values contain escape character [\#1533](https://github.com/delta-io/delta-rs/issues/1533) +- add documentation how to Query Delta natively from datafusion [\#1485](https://github.com/delta-io/delta-rs/issues/1485) +- Python: write\_deltalake to ADLS Gen2 issue [\#1456](https://github.com/delta-io/delta-rs/issues/1456) +- Partition values that have been url encoded cannot be read when using deltalake [\#1446](https://github.com/delta-io/delta-rs/issues/1446) +- Error optimizing large table [\#1419](https://github.com/delta-io/delta-rs/issues/1419) +- Cannot read partitions with special characters \(including space\) with pyarrow \>= 11 [\#1393](https://github.com/delta-io/delta-rs/issues/1393) +- ImportError: deltalake/\_internal.abi3.so: cannot allocate memory in static TLS block [\#1380](https://github.com/delta-io/delta-rs/issues/1380) +- Invalid JSON in log record missing field `schemaString` for DLT tables [\#1302](https://github.com/delta-io/delta-rs/issues/1302) +- Special characters in partition path not handled locally [\#1299](https://github.com/delta-io/delta-rs/issues/1299) + +**Merged pull requests:** + +- chore: bump rust crate version [\#1675](https://github.com/delta-io/delta-rs/pull/1675) ([rtyler](https://github.com/rtyler)) +- fix: change partitioning schema from large to normal string for pyarrow\<12 [\#1671](https://github.com/delta-io/delta-rs/pull/1671) ([ion-elgreco](https://github.com/ion-elgreco)) +- feat: allow to set large dtypes for the schema check in `write_deltalake` [\#1668](https://github.com/delta-io/delta-rs/pull/1668) ([ion-elgreco](https://github.com/ion-elgreco)) +- docs: small consistency update in guide and readme [\#1666](https://github.com/delta-io/delta-rs/pull/1666) ([ion-elgreco](https://github.com/ion-elgreco)) +- fix: exception string in writer.py [\#1665](https://github.com/delta-io/delta-rs/pull/1665) ([sebdiem](https://github.com/sebdiem)) +- chore: increment python library version [\#1664](https://github.com/delta-io/delta-rs/pull/1664) ([wjones127](https://github.com/wjones127)) +- docs: fix some typos [\#1662](https://github.com/delta-io/delta-rs/pull/1662) ([ion-elgreco](https://github.com/ion-elgreco)) +- fix: more consistent handling of partition values and file paths [\#1661](https://github.com/delta-io/delta-rs/pull/1661) ([roeap](https://github.com/roeap)) +- docs: add docstring to protocol method [\#1660](https://github.com/delta-io/delta-rs/pull/1660) ([MrPowers](https://github.com/MrPowers)) +- docs: make docs.rs build docs with all features enabled [\#1658](https://github.com/delta-io/delta-rs/pull/1658) ([simonvandel](https://github.com/simonvandel)) +- fix: enable offset listing for s3 [\#1654](https://github.com/delta-io/delta-rs/pull/1654) ([eeroel](https://github.com/eeroel)) +- chore: fix the incorrect Slack link in our readme [\#1649](https://github.com/delta-io/delta-rs/pull/1649) ([rtyler](https://github.com/rtyler)) +- fix: compensate for invalid log files created by Delta Live Tables [\#1647](https://github.com/delta-io/delta-rs/pull/1647) ([rtyler](https://github.com/rtyler)) +- chore: proposed updated CODEOWNERS to allow better review notifications [\#1646](https://github.com/delta-io/delta-rs/pull/1646) ([rtyler](https://github.com/rtyler)) +- feat: expose min\_commit\_interval to `optimize.compact` and `optimize.z_order` [\#1645](https://github.com/delta-io/delta-rs/pull/1645) ([ion-elgreco](https://github.com/ion-elgreco)) +- fix: avoid excess listing of log files [\#1644](https://github.com/delta-io/delta-rs/pull/1644) ([eeroel](https://github.com/eeroel)) +- fix: introduce support for Microsoft OneLake [\#1642](https://github.com/delta-io/delta-rs/pull/1642) ([rtyler](https://github.com/rtyler)) +- fix: explicitly require chrono 0.4.31 or greater [\#1641](https://github.com/delta-io/delta-rs/pull/1641) ([rtyler](https://github.com/rtyler)) +- fix: include in-progress row group when calculating in-memory buffer length [\#1638](https://github.com/delta-io/delta-rs/pull/1638) ([BnMcG](https://github.com/BnMcG)) +- chore: relax chrono pin to 0.4 [\#1635](https://github.com/delta-io/delta-rs/pull/1635) ([houqp](https://github.com/houqp)) +- chore: update datafusion to 31, arrow to 46 and object\_store to 0.7 [\#1634](https://github.com/delta-io/delta-rs/pull/1634) ([houqp](https://github.com/houqp)) +- docs: update Readme [\#1633](https://github.com/delta-io/delta-rs/pull/1633) ([dennyglee](https://github.com/dennyglee)) +- chore: pin the chrono dependency [\#1631](https://github.com/delta-io/delta-rs/pull/1631) ([rtyler](https://github.com/rtyler)) +- feat: pass known file sizes to filesystem in Python [\#1630](https://github.com/delta-io/delta-rs/pull/1630) ([eeroel](https://github.com/eeroel)) +- feat: implement parsing for the new `domainMetadata` actions in the commit log [\#1629](https://github.com/delta-io/delta-rs/pull/1629) ([rtyler](https://github.com/rtyler)) +- ci: fix python release [\#1624](https://github.com/delta-io/delta-rs/pull/1624) ([wjones127](https://github.com/wjones127)) +- ci: extend azure timeout [\#1622](https://github.com/delta-io/delta-rs/pull/1622) ([wjones127](https://github.com/wjones127)) +- feat: allow multiple incremental commits in optimize [\#1621](https://github.com/delta-io/delta-rs/pull/1621) ([kvap](https://github.com/kvap)) +- fix: change map nullable value to false [\#1620](https://github.com/delta-io/delta-rs/pull/1620) ([cmackenzie1](https://github.com/cmackenzie1)) +- Introduce the changelog for the last couple releases [\#1617](https://github.com/delta-io/delta-rs/pull/1617) ([rtyler](https://github.com/rtyler)) +- chore: bump python version to 0.10.2 [\#1616](https://github.com/delta-io/delta-rs/pull/1616) ([wjones127](https://github.com/wjones127)) +- perf: avoid holding GIL in DeltaFileSystemHandler [\#1615](https://github.com/delta-io/delta-rs/pull/1615) ([wjones127](https://github.com/wjones127)) +- fix: don't re-encode paths [\#1613](https://github.com/delta-io/delta-rs/pull/1613) ([wjones127](https://github.com/wjones127)) +- feat: use url parsing from object store [\#1592](https://github.com/delta-io/delta-rs/pull/1592) ([roeap](https://github.com/roeap)) +- feat: buffered reading of transaction logs [\#1549](https://github.com/delta-io/delta-rs/pull/1549) ([eeroel](https://github.com/eeroel)) +- feat: merge operation [\#1522](https://github.com/delta-io/delta-rs/pull/1522) ([Blajda](https://github.com/Blajda)) +- feat: expose create\_checkpoint\_for to the public [\#1514](https://github.com/delta-io/delta-rs/pull/1514) ([haruband](https://github.com/haruband)) +- docs: update Readme [\#1440](https://github.com/delta-io/delta-rs/pull/1440) ([roeap](https://github.com/roeap)) +- refactor: re-organize top level modules [\#1434](https://github.com/delta-io/delta-rs/pull/1434) ([roeap](https://github.com/roeap)) +- feat: integrate unity catalog with datafusion [\#1338](https://github.com/delta-io/delta-rs/pull/1338) ([roeap](https://github.com/roeap)) + ## [rust-v0.15.0](https://github.com/delta-io/delta-rs/tree/rust-v0.15.0) (2023-09-06) [Full Changelog](https://github.com/delta-io/delta-rs/compare/rust-v0.14.0...rust-v0.15.0) diff --git a/Cargo.toml b/Cargo.toml index 3015ce4bdd..0b3862bd1f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,10 @@ [workspace] -members = ["rust", "python"] -exclude = ["proofs", "delta-inspect"] +members = [ + "crates/*", + "delta-inspect", + "python", +] +exclude = ["proofs"] resolver = "2" [profile.release-with-debug] @@ -15,23 +19,23 @@ debug = "line-tables-only" [workspace.dependencies] # arrow -arrow = { version = "46" } -arrow-array = { version = "46" } -arrow-buffer = { version = "46" } -arrow-cast = { version = "46" } -arrow-ord = { version = "46" } -arrow-row = { version = "46" } -arrow-schema = { version = "46" } -arrow-select = { version = "46" } -parquet = { version = "46" } +arrow = { version = "47" } +arrow-array = { version = "47" } +arrow-buffer = { version = "47" } +arrow-cast = { version = "47" } +arrow-ord = { version = "47" } +arrow-row = { version = "47" } +arrow-schema = { version = "47" } +arrow-select = { version = "47" } +parquet = { version = "47" } # datafusion -datafusion = { version = "31" } -datafusion-expr = { version = "31" } -datafusion-common = { version = "31" } -datafusion-proto = { version = "31" } -datafusion-sql = { version = "31" } -datafusion-physical-expr = { version = "31" } +datafusion = { version = "32" } +datafusion-expr = { version = "32" } +datafusion-common = { version = "32" } +datafusion-proto = { version = "32" } +datafusion-sql = { version = "32" } +datafusion-physical-expr = { version = "32" } # serde serde = { version = "1", features = ["derive"] } diff --git a/README.md b/README.md index 09d8fa4753..b3dd824b77 100644 --- a/README.md +++ b/README.md @@ -32,7 +32,7 @@ Deltalake - + #delta-rs in the Delta Lake Slack workspace

@@ -57,7 +57,7 @@ API that lets you query, inspect, and operate your Delta Lake with ease. - [Quick Start](#quick-start) - [Get Involved](#get-involved) -- [Integartions](#integrations) +- [Integrations](#integrations) - [Features](#features) ## Quick Start @@ -66,8 +66,7 @@ The `deltalake` library aims to adopt patterns from other libraries in data proc so getting started should look familiar. ```py3 -from deltalake import DeltaTable -from deltalake.write import write_deltalake +from deltalake import DeltaTable, write_deltalake import pandas as pd # write some data into a delta table @@ -103,10 +102,10 @@ You can also try Delta Lake docker at [DockerHub](https://go.delta.io/dockerhub) ## Get Involved -We encourage you to reach out, and are [commited](https://github.com/delta-io/delta-rs/blob/main/CODE_OF_CONDUCT.md) +We encourage you to reach out, and are [committed](https://github.com/delta-io/delta-rs/blob/main/CODE_OF_CONDUCT.md) to provide a welcoming community. -- [Join us in our Slack workspace](https://go.delta.io/slack) +- [Join us in our Slack workspace](https://join.slack.com/t/delta-users/shared_invite/zt-23h0xwez7-wDTm43ZVEW2ZcbKn6Bc8Fg) - [Report an issue](https://github.com/delta-io/delta-rs/issues/new?template=bug_report.md) - Looking to contribute? See our [good first issues](https://github.com/delta-io/delta-rs/contribute). @@ -139,28 +138,28 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc | S3 - R2 | ![done] | ![done] | requires lock for concurrent writes | | Azure Blob | ![done] | ![done] | | | Azure ADLS Gen2 | ![done] | ![done] | | -| Micorosft OneLake | [![open]][onelake-rs] | [![open]][onelake-rs] | | +| 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] | | 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 | [![open]][merge-rs] | [![open]][merge-py] | | -| FS check | ![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 | [![semi-done]][merge-rs] | [![open]][merge-py] | Merge two tables (limited to full re-write) | +| FS check | ![done] | | Remove corrupted files from table | ### Protocol Support Level | Writer Version | Requirement | Status | | -------------- | --------------------------------------------- | :------------------: | -| Version 2 | Append Only Tables | [![open]][roadmap] | +| 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] | @@ -173,13 +172,14 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc | Reader Version | Requirement | Status | | -------------- | ----------------------------------- | ------ | -| Version 2 | Collumn Mapping | | +| Version 2 | Column Mapping | | | Version 3 | Table Features (requires reader V7) | | [datafusion]: https://github.com/apache/arrow-datafusion [ballista]: https://github.com/apache/arrow-ballista [polars]: https://github.com/pola-rs/polars [open]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/IssueNeutral.svg +[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 diff --git a/crates/README.md b/crates/README.md new file mode 100644 index 0000000000..d5b17317ca --- /dev/null +++ b/crates/README.md @@ -0,0 +1,3 @@ +# Delta Lake Rust crates + +This directory contains all of the crates published by the [delta-rs](https://github.com/delta-io/delta-rs) project. These crates were originally split based on the proposal in [#1713](https://github.com/delta-io/delta-rs/discussions/1713). diff --git a/rust/.gitignore b/crates/deltalake-core/.gitignore similarity index 100% rename from rust/.gitignore rename to crates/deltalake-core/.gitignore diff --git a/rust/.ignore b/crates/deltalake-core/.ignore similarity index 100% rename from rust/.ignore rename to crates/deltalake-core/.ignore diff --git a/rust/Cargo.toml b/crates/deltalake-core/Cargo.toml similarity index 90% rename from rust/Cargo.toml rename to crates/deltalake-core/Cargo.toml index 72ad44bbac..e645b6bfd0 100644 --- a/rust/Cargo.toml +++ b/crates/deltalake-core/Cargo.toml @@ -1,6 +1,6 @@ [package] -name = "deltalake" -version = "0.15.0" +name = "deltalake-core" +version = "0.17.0" rust-version = "1.64" authors = ["Qingping Hou "] homepage = "https://github.com/delta-io/delta.rs" @@ -12,6 +12,11 @@ repository = "https://github.com/delta-io/delta.rs" readme = "README.md" edition = "2021" +[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", "glue", "hdfs", "json", "python", "s3", "unity-experimental"] + [dependencies] # arrow arrow = { workspace = true, optional = true } @@ -20,7 +25,7 @@ arrow-buffer = { workspace = true, optional = true } arrow-cast = { workspace = true, optional = true } arrow-ord = { workspace = true, optional = true } arrow-row = { workspace = true, optional = true } -arrow-schema = { workspace = true, optional = true } +arrow-schema = { workspace = true, optional = true, features = ["serde"] } arrow-select = { workspace = true, optional = true } parquet = { workspace = true, features = [ "async", @@ -102,10 +107,10 @@ reqwest = { version = "0.11.18", default-features = false, features = [ # Datafusion dashmap = { version = "5", optional = true } -sqlparser = { version = "0.37", optional = true } +sqlparser = { version = "0.38", optional = true } # NOTE dependencies only for integration tests -fs_extra = { version = "1.2.0", optional = true } +fs_extra = { version = "1.3.0", optional = true } tempdir = { version = "0", optional = true } dynamodb_lock = { version = "0", default-features = false, optional = true } @@ -177,11 +182,3 @@ unity-experimental = ["reqwest", "tracing", "hyper"] [[bench]] name = "read_checkpoint" harness = false - -[[example]] -name = "basic_operations" -required-features = ["datafusion"] - -[[example]] -name = "recordbatch-writer" -required-features = ["arrow"] diff --git a/rust/README.md b/crates/deltalake-core/README.md similarity index 87% rename from rust/README.md rename to crates/deltalake-core/README.md index 659de48566..b251148c69 100644 --- a/rust/README.md +++ b/crates/deltalake-core/README.md @@ -16,8 +16,11 @@ println!("{}", table.get_files()); ### CLI +Navigate into the `delta-inspect` directory first and run the following command +Please noted that the test data is under `rust` instead of `delta-inspect` + ```bash -❯ cargo run --bin delta-inspect files ./tests/data/delta-0.2.0 +❯ cargo run --bin delta-inspect files ../rust/tests/data/delta-0.2.0 part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet @@ -33,7 +36,7 @@ DeltaTable(./tests/data/delta-0.2.0) The examples folder shows how to use Rust API to manipulate Delta tables. -Examples can be run using the `cargo run --example` command. For example: +Navigate into the `rust` directory first and examples can be run using the `cargo run --example` command. For example: ```bash cargo run --example read_delta_table diff --git a/rust/benches/read_checkpoint.rs b/crates/deltalake-core/benches/read_checkpoint.rs similarity index 91% rename from rust/benches/read_checkpoint.rs rename to crates/deltalake-core/benches/read_checkpoint.rs index 9824f15eb0..2ecbee661b 100644 --- a/rust/benches/read_checkpoint.rs +++ b/crates/deltalake-core/benches/read_checkpoint.rs @@ -1,6 +1,6 @@ use criterion::{criterion_group, criterion_main, Criterion}; -use deltalake::delta::DeltaTableConfig; -use deltalake::table_state::DeltaTableState; +use deltalake::table::state::DeltaTableState; +use deltalake::DeltaTableConfig; use std::fs::File; use std::io::Read; diff --git a/rust/src/data_catalog/client/backoff.rs b/crates/deltalake-core/src/data_catalog/client/backoff.rs similarity index 100% rename from rust/src/data_catalog/client/backoff.rs rename to crates/deltalake-core/src/data_catalog/client/backoff.rs diff --git a/rust/src/data_catalog/client/mock_server.rs b/crates/deltalake-core/src/data_catalog/client/mock_server.rs similarity index 100% rename from rust/src/data_catalog/client/mock_server.rs rename to crates/deltalake-core/src/data_catalog/client/mock_server.rs diff --git a/rust/src/data_catalog/client/mod.rs b/crates/deltalake-core/src/data_catalog/client/mod.rs similarity index 100% rename from rust/src/data_catalog/client/mod.rs rename to crates/deltalake-core/src/data_catalog/client/mod.rs diff --git a/rust/src/data_catalog/client/pagination.rs b/crates/deltalake-core/src/data_catalog/client/pagination.rs similarity index 100% rename from rust/src/data_catalog/client/pagination.rs rename to crates/deltalake-core/src/data_catalog/client/pagination.rs diff --git a/rust/src/data_catalog/client/retry.rs b/crates/deltalake-core/src/data_catalog/client/retry.rs similarity index 100% rename from rust/src/data_catalog/client/retry.rs rename to crates/deltalake-core/src/data_catalog/client/retry.rs diff --git a/rust/src/data_catalog/client/token.rs b/crates/deltalake-core/src/data_catalog/client/token.rs similarity index 100% rename from rust/src/data_catalog/client/token.rs rename to crates/deltalake-core/src/data_catalog/client/token.rs diff --git a/rust/src/data_catalog/glue/mod.rs b/crates/deltalake-core/src/data_catalog/glue/mod.rs similarity index 100% rename from rust/src/data_catalog/glue/mod.rs rename to crates/deltalake-core/src/data_catalog/glue/mod.rs diff --git a/rust/src/data_catalog/mod.rs b/crates/deltalake-core/src/data_catalog/mod.rs similarity index 100% rename from rust/src/data_catalog/mod.rs rename to crates/deltalake-core/src/data_catalog/mod.rs diff --git a/rust/src/data_catalog/storage/mod.rs b/crates/deltalake-core/src/data_catalog/storage/mod.rs similarity index 96% rename from rust/src/data_catalog/storage/mod.rs rename to crates/deltalake-core/src/data_catalog/storage/mod.rs index 726afee102..f645d370c1 100644 --- a/rust/src/data_catalog/storage/mod.rs +++ b/crates/deltalake-core/src/data_catalog/storage/mod.rs @@ -13,8 +13,9 @@ use futures::TryStreamExt; use object_store::ObjectStore; use crate::errors::DeltaResult; +use crate::open_table_with_storage_options; use crate::storage::config::{configure_store, StorageOptions}; -use crate::{ensure_table_uri, open_table_with_storage_options}; +use crate::table::builder::ensure_table_uri; const DELTA_LOG_FOLDER: &str = "_delta_log"; @@ -46,9 +47,9 @@ impl ListingSchemaProvider { storage_options: Option>, ) -> DeltaResult { let uri = ensure_table_uri(root_uri)?; - let storage_options = storage_options.unwrap_or_default().into(); + let mut storage_options = storage_options.unwrap_or_default().into(); // We already parsed the url, so unwrapping is safe. - let store = configure_store(&uri, &storage_options)?; + let store = configure_store(&uri, &mut storage_options)?; Ok(Self { authority: uri.to_string(), store, diff --git a/rust/src/data_catalog/unity/credential.rs b/crates/deltalake-core/src/data_catalog/unity/credential.rs similarity index 100% rename from rust/src/data_catalog/unity/credential.rs rename to crates/deltalake-core/src/data_catalog/unity/credential.rs diff --git a/rust/src/data_catalog/unity/datafusion.rs b/crates/deltalake-core/src/data_catalog/unity/datafusion.rs similarity index 100% rename from rust/src/data_catalog/unity/datafusion.rs rename to crates/deltalake-core/src/data_catalog/unity/datafusion.rs diff --git a/rust/src/data_catalog/unity/mod.rs b/crates/deltalake-core/src/data_catalog/unity/mod.rs similarity index 96% rename from rust/src/data_catalog/unity/mod.rs rename to crates/deltalake-core/src/data_catalog/unity/mod.rs index 3110f07020..e9de725923 100644 --- a/rust/src/data_catalog/unity/mod.rs +++ b/crates/deltalake-core/src/data_catalog/unity/mod.rs @@ -84,16 +84,27 @@ pub enum UnityCatalogConfigKey { /// - `unity_workspace_url` /// - `databricks_workspace_url` /// - `workspace_url` + #[deprecated(since = "0.17.0", note = "Please use the DATABRICKS_HOST env variable")] WorkspaceUrl, + /// Host of the Databricks workspace + Host, + /// Access token to authorize API requests /// /// Supported keys: /// - `unity_access_token` /// - `databricks_access_token` /// - `access_token` + #[deprecated( + since = "0.17.0", + note = "Please use the DATABRICKS_TOKEN env variable" + )] AccessToken, + /// Token to use for Databricks Unity + Token, + /// Service principal client id for authorizing requests /// /// Supported keys: @@ -167,6 +178,7 @@ pub enum UnityCatalogConfigKey { impl FromStr for UnityCatalogConfigKey { type Err = DataCatalogError; + #[allow(deprecated)] fn from_str(s: &str) -> Result { match s { "access_token" | "unity_access_token" | "databricks_access_token" => { @@ -187,6 +199,7 @@ impl FromStr for UnityCatalogConfigKey { "federated_token_file" | "unity_federated_token_file" | "databricks_federated_token_file" => Ok(UnityCatalogConfigKey::FederatedTokenFile), + "host" => Ok(UnityCatalogConfigKey::Host), "msi_endpoint" | "unity_msi_endpoint" | "databricks_msi_endpoint" => { Ok(UnityCatalogConfigKey::MsiEndpoint) } @@ -196,6 +209,7 @@ impl FromStr for UnityCatalogConfigKey { "object_id" | "unity_object_id" | "databricks_object_id" => { Ok(UnityCatalogConfigKey::ObjectId) } + "token" => Ok(UnityCatalogConfigKey::Token), "use_azure_cli" | "unity_use_azure_cli" | "databricks_use_azure_cli" => { Ok(UnityCatalogConfigKey::UseAzureCli) } @@ -207,6 +221,7 @@ impl FromStr for UnityCatalogConfigKey { } } +#[allow(deprecated)] impl AsRef for UnityCatalogConfigKey { fn as_ref(&self) -> &str { match self { @@ -216,10 +231,12 @@ impl AsRef for UnityCatalogConfigKey { UnityCatalogConfigKey::ClientId => "unity_client_id", UnityCatalogConfigKey::ClientSecret => "unity_client_secret", UnityCatalogConfigKey::FederatedTokenFile => "unity_federated_token_file", + UnityCatalogConfigKey::Host => "databricks_host", UnityCatalogConfigKey::MsiEndpoint => "unity_msi_endpoint", UnityCatalogConfigKey::MsiResourceId => "unity_msi_resource_id", UnityCatalogConfigKey::ObjectId => "unity_object_id", UnityCatalogConfigKey::UseAzureCli => "unity_use_azure_cli", + UnityCatalogConfigKey::Token => "databricks_token", UnityCatalogConfigKey::WorkspaceUrl => "unity_workspace_url", } } @@ -268,6 +285,7 @@ pub struct UnityCatalogBuilder { client_options: super::client::ClientOptions, } +#[allow(deprecated)] impl UnityCatalogBuilder { /// Create a new [`UnityCatalogBuilder`] with default values. pub fn new() -> Self { @@ -281,19 +299,21 @@ impl UnityCatalogBuilder { value: impl Into, ) -> DataCatalogResult { match UnityCatalogConfigKey::from_str(key.as_ref())? { - UnityCatalogConfigKey::WorkspaceUrl => self.workspace_url = Some(value.into()), UnityCatalogConfigKey::AccessToken => self.bearer_token = Some(value.into()), UnityCatalogConfigKey::ClientId => self.client_id = Some(value.into()), UnityCatalogConfigKey::ClientSecret => self.client_secret = Some(value.into()), UnityCatalogConfigKey::AuthorityId => self.authority_id = Some(value.into()), UnityCatalogConfigKey::AuthorityHost => self.authority_host = Some(value.into()), + UnityCatalogConfigKey::Host => self.workspace_url = Some(value.into()), UnityCatalogConfigKey::MsiEndpoint => self.msi_endpoint = Some(value.into()), UnityCatalogConfigKey::ObjectId => self.object_id = Some(value.into()), UnityCatalogConfigKey::MsiResourceId => self.msi_resource_id = Some(value.into()), UnityCatalogConfigKey::FederatedTokenFile => { self.federated_token_file = Some(value.into()) } + UnityCatalogConfigKey::Token => self.bearer_token = Some(value.into()), UnityCatalogConfigKey::UseAzureCli => self.use_azure_cli = str_is_truthy(&value.into()), + UnityCatalogConfigKey::WorkspaceUrl => self.workspace_url = Some(value.into()), }; Ok(self) } diff --git a/rust/src/data_catalog/unity/models.rs b/crates/deltalake-core/src/data_catalog/unity/models.rs similarity index 100% rename from rust/src/data_catalog/unity/models.rs rename to crates/deltalake-core/src/data_catalog/unity/models.rs diff --git a/crates/deltalake-core/src/delta_datafusion/expr.rs b/crates/deltalake-core/src/delta_datafusion/expr.rs new file mode 100644 index 0000000000..815b01831f --- /dev/null +++ b/crates/deltalake-core/src/delta_datafusion/expr.rs @@ -0,0 +1,582 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// +// This product includes software from the Datafusion project (Apache 2.0) +// https://github.com/apache/arrow-datafusion +// Display functions and required macros were pulled from https://github.com/apache/arrow-datafusion/blob/ddb95497e2792015d5a5998eec79aac8d37df1eb/datafusion/expr/src/expr.rs + +//! Utility functions for Datafusion's Expressions + +use std::{ + fmt::{self, Display, Formatter, Write}, + sync::Arc, +}; + +use arrow_schema::DataType; +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, +}; +use datafusion_sql::planner::{ContextProvider, SqlToRel}; +use sqlparser::ast::escape_quoted_string; +use sqlparser::dialect::GenericDialect; +use sqlparser::parser::Parser; +use sqlparser::tokenizer::Tokenizer; + +use crate::{DeltaResult, DeltaTableError}; + +pub(crate) struct DeltaContextProvider<'a> { + state: &'a SessionState, +} + +impl<'a> ContextProvider for DeltaContextProvider<'a> { + fn get_table_provider(&self, _name: TableReference) -> DFResult> { + unimplemented!() + } + + fn get_function_meta(&self, name: &str) -> Option> { + self.state.scalar_functions().get(name).cloned() + } + + fn get_aggregate_meta(&self, name: &str) -> Option> { + self.state.aggregate_functions().get(name).cloned() + } + + fn get_variable_type(&self, _var: &[String]) -> Option { + unimplemented!() + } + + fn options(&self) -> &ConfigOptions { + self.state.config_options() + } + + fn get_window_meta(&self, name: &str) -> Option> { + self.state.window_functions().get(name).cloned() + } +} + +/// Parse a string predicate into an `Expr` +pub(crate) fn parse_predicate_expression( + schema: &DFSchema, + expr: impl AsRef, + df_state: &SessionState, +) -> DeltaResult { + let dialect = &GenericDialect {}; + let mut tokenizer = Tokenizer::new(dialect, expr.as_ref()); + let tokens = tokenizer + .tokenize() + .map_err(|err| DeltaTableError::GenericError { + source: Box::new(err), + })?; + let sql = Parser::new(dialect) + .with_tokens(tokens) + .parse_expr() + .map_err(|err| DeltaTableError::GenericError { + source: Box::new(err), + })?; + + let context_provider = DeltaContextProvider { state: df_state }; + let sql_to_rel = SqlToRel::new(&context_provider); + + Ok(sql_to_rel.sql_to_expr(sql, schema, &mut Default::default())?) +} + +struct SqlFormat<'a> { + expr: &'a Expr, +} + +macro_rules! expr_vec_fmt { + ( $ARRAY:expr ) => {{ + $ARRAY + .iter() + .map(|e| format!("{}", SqlFormat { expr: e })) + .collect::>() + .join(", ") + }}; +} + +struct BinaryExprFormat<'a> { + expr: &'a BinaryExpr, +} + +impl<'a> Display for BinaryExprFormat<'a> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + // Put parentheses around child binary expressions so that we can see the difference + // between `(a OR b) AND c` and `a OR (b AND c)`. We only insert parentheses when needed, + // based on operator precedence. For example, `(a AND b) OR c` and `a AND b OR c` are + // equivalent and the parentheses are not necessary. + + fn write_child(f: &mut Formatter<'_>, expr: &Expr, precedence: u8) -> fmt::Result { + match expr { + Expr::BinaryExpr(child) => { + let p = child.op.precedence(); + if p == 0 || p < precedence { + write!(f, "({})", BinaryExprFormat { expr: child })?; + } else { + write!(f, "{}", BinaryExprFormat { expr: child })?; + } + } + _ => write!(f, "{}", SqlFormat { expr })?, + } + Ok(()) + } + + let precedence = self.expr.op.precedence(); + write_child(f, self.expr.left.as_ref(), precedence)?; + write!(f, " {} ", self.expr.op)?; + write_child(f, self.expr.right.as_ref(), precedence) + } +} + +impl<'a> Display for SqlFormat<'a> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self.expr { + Expr::Column(c) => write!(f, "{c}"), + Expr::Literal(v) => write!(f, "{}", ScalarValueFormat { scalar: v }), + Expr::Case(case) => { + write!(f, "CASE ")?; + if let Some(e) = &case.expr { + write!(f, "{} ", SqlFormat { expr: e })?; + } + for (w, t) in &case.when_then_expr { + write!( + f, + "WHEN {} THEN {} ", + SqlFormat { expr: w }, + SqlFormat { expr: t } + )?; + } + if let Some(e) = &case.else_expr { + write!(f, "ELSE {} ", SqlFormat { expr: e })?; + } + write!(f, "END") + } + Expr::Not(expr) => write!(f, "NOT {}", SqlFormat { expr }), + Expr::Negative(expr) => write!(f, "(- {})", SqlFormat { expr }), + Expr::IsNull(expr) => write!(f, "{} IS NULL", SqlFormat { expr }), + Expr::IsNotNull(expr) => write!(f, "{} IS NOT NULL", SqlFormat { expr }), + Expr::IsTrue(expr) => write!(f, "{} IS TRUE", SqlFormat { expr }), + Expr::IsFalse(expr) => write!(f, "{} IS FALSE", SqlFormat { expr }), + Expr::IsUnknown(expr) => write!(f, "{} IS UNKNOWN", SqlFormat { expr }), + Expr::IsNotTrue(expr) => write!(f, "{} IS NOT TRUE", SqlFormat { expr }), + 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::Cast(Cast { expr, data_type }) => { + write!(f, "arrow_cast({}, '{}')", SqlFormat { expr }, data_type) + } + Expr::Between(Between { + expr, + negated, + low, + high, + }) => { + if *negated { + write!( + f, + "{} NOT BETWEEN {} AND {}", + SqlFormat { expr }, + SqlFormat { expr: low }, + SqlFormat { expr: high } + ) + } else { + write!( + f, + "{} BETWEEN {} AND {}", + SqlFormat { expr }, + SqlFormat { expr: low }, + SqlFormat { expr: high } + ) + } + } + Expr::Like(Like { + negated, + expr, + pattern, + escape_char, + case_insensitive, + }) => { + write!(f, "{}", SqlFormat { expr })?; + let op_name = if *case_insensitive { "ILIKE" } else { "LIKE" }; + if *negated { + write!(f, " NOT")?; + } + if let Some(char) = escape_char { + write!( + f, + " {op_name} {} ESCAPE '{char}'", + SqlFormat { expr: pattern } + ) + } else { + write!(f, " {op_name} {}", SqlFormat { expr: pattern }) + } + } + Expr::SimilarTo(Like { + negated, + expr, + pattern, + escape_char, + case_insensitive: _, + }) => { + write!(f, "{expr}")?; + if *negated { + write!(f, " NOT")?; + } + if let Some(char) = escape_char { + write!(f, " SIMILAR TO {pattern} ESCAPE '{char}'") + } else { + write!(f, " SIMILAR TO {pattern}") + } + } + Expr::InList(InList { + expr, + list, + negated, + }) => { + if *negated { + write!(f, "{expr} NOT IN ({})", expr_vec_fmt!(list)) + } else { + write!(f, "{expr} IN ({})", expr_vec_fmt!(list)) + } + } + _ => Err(fmt::Error), + } + } +} + +/// Format an `Expr` to a parsable SQL expression +pub fn fmt_expr_to_sql(expr: &Expr) -> Result { + let mut s = String::new(); + write!(&mut s, "{}", SqlFormat { expr }).map_err(|_| { + DeltaTableError::Generic("Unable to convert expression to string".to_owned()) + })?; + Ok(s) +} + +fn fmt_function(f: &mut fmt::Formatter, fun: &str, distinct: bool, args: &[Expr]) -> fmt::Result { + let args: Vec = args + .iter() + .map(|arg| format!("{}", SqlFormat { expr: arg })) + .collect(); + + let distinct_str = match distinct { + true => "DISTINCT ", + false => "", + }; + write!(f, "{}({}{})", fun, distinct_str, args.join(", ")) +} + +macro_rules! format_option { + ($F:expr, $EXPR:expr) => {{ + match $EXPR { + Some(e) => write!($F, "{e}"), + None => write!($F, "NULL"), + } + }}; +} + +struct ScalarValueFormat<'a> { + scalar: &'a ScalarValue, +} + +impl<'a> fmt::Display for ScalarValueFormat<'a> { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self.scalar { + ScalarValue::Boolean(e) => format_option!(f, e)?, + ScalarValue::Float32(e) => format_option!(f, e)?, + ScalarValue::Float64(e) => format_option!(f, e)?, + ScalarValue::Int8(e) => format_option!(f, e)?, + ScalarValue::Int16(e) => format_option!(f, e)?, + ScalarValue::Int32(e) => format_option!(f, e)?, + ScalarValue::Int64(e) => format_option!(f, e)?, + ScalarValue::UInt8(e) => format_option!(f, e)?, + ScalarValue::UInt16(e) => format_option!(f, e)?, + ScalarValue::UInt32(e) => format_option!(f, e)?, + ScalarValue::UInt64(e) => format_option!(f, e)?, + ScalarValue::Utf8(e) | ScalarValue::LargeUtf8(e) => match e { + Some(e) => write!(f, "'{}'", escape_quoted_string(e, '\''))?, + None => write!(f, "NULL")?, + }, + ScalarValue::Binary(e) + | ScalarValue::FixedSizeBinary(_, e) + | ScalarValue::LargeBinary(e) => match e { + Some(l) => write!( + f, + "decode('{}', 'hex')", + l.iter() + .map(|v| format!("{v:02x}")) + .collect::>() + .join("") + )?, + None => write!(f, "NULL")?, + }, + ScalarValue::Null => write!(f, "NULL")?, + _ => return Err(fmt::Error), + }; + Ok(()) + } +} + +#[cfg(test)] +mod test { + use std::collections::HashMap; + + use arrow_schema::DataType; + use datafusion::prelude::SessionContext; + use datafusion_common::{DFSchema, ScalarValue}; + use datafusion_expr::{col, decode, lit, substring, Cast, Expr, ExprSchemable}; + + use crate::{DeltaOps, DeltaTable, Schema, SchemaDataType, SchemaField}; + + use super::fmt_expr_to_sql; + + struct ParseTest { + expr: Expr, + expected: String, + override_expected_expr: Option, + } + + macro_rules! simple { + ( $EXPR:expr, $STR:expr ) => {{ + ParseTest { + expr: $EXPR, + expected: $STR, + override_expected_expr: None, + } + }}; + } + + async fn setup_table() -> DeltaTable { + let schema = Schema::new(vec![ + SchemaField::new( + "id".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "value".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "value2".to_string(), + SchemaDataType::primitive("integer".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "modified".to_string(), + SchemaDataType::primitive("string".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "active".to_string(), + SchemaDataType::primitive("boolean".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "money".to_string(), + SchemaDataType::primitive("decimal(12,2)".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "_date".to_string(), + SchemaDataType::primitive("date".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "_timestamp".to_string(), + SchemaDataType::primitive("timestamp".to_string()), + true, + HashMap::new(), + ), + SchemaField::new( + "_binary".to_string(), + SchemaDataType::primitive("binary".to_string()), + true, + HashMap::new(), + ), + ]); + + let table = DeltaOps::new_in_memory() + .create() + .with_columns(schema.get_fields().clone()) + .await + .unwrap(); + assert_eq!(table.version(), 0); + table + } + + #[tokio::test] + async fn test_expr_sql() { + let table = setup_table().await; + + // String expression that we output must be parsable for conflict resolution. + let tests = vec![ + simple!( + Expr::Cast(Cast { + expr: Box::new(lit(1_i64)), + data_type: DataType::Int32 + }), + "arrow_cast(1, 'Int32')".to_string() + ), + simple!(col("value").eq(lit(3_i64)), "value = 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()), + simple!(col("active").is_null(), "active IS NULL".to_string()), + simple!( + col("modified").eq(lit("2021-02-03")), + "modified = '2021-02-03'".to_string() + ), + simple!( + col("modified").eq(lit("'validate ' escapi\\ng'")), + "modified = '''validate '' escapi\\ng'''".to_string() + ), + simple!(col("money").gt(lit(0.10)), "money > 0.1".to_string()), + ParseTest { + expr: col("_binary").eq(lit(ScalarValue::Binary(Some(vec![0xAA, 0x00, 0xFF])))), + expected: "_binary = decode('aa00ff', 'hex')".to_string(), + override_expected_expr: Some(col("_binary").eq(decode(lit("aa00ff"), lit("hex")))), + }, + simple!( + col("value").between(lit(20_i64), lit(30_i64)), + "value BETWEEN 20 AND 30".to_string() + ), + simple!( + col("value").not_between(lit(20_i64), lit(30_i64)), + "value NOT BETWEEN 20 AND 30".to_string() + ), + simple!( + col("modified").like(lit("abc%")), + "modified LIKE 'abc%'".to_string() + ), + simple!( + col("modified").not_like(lit("abc%")), + "modified NOT LIKE 'abc%'".to_string() + ), + simple!( + (((col("value") * lit(2_i64) + col("value2")) / lit(3_i64)) - col("value")) + .gt(lit(0_i64)), + "(value * 2 + value2) / 3 - value > 0".to_string() + ), + simple!( + col("modified").in_list(vec![lit("a"), lit("c")], false), + "modified IN ('a', 'c')".to_string() + ), + simple!( + col("modified").in_list(vec![lit("a"), lit("c")], true), + "modified NOT IN ('a', 'c')".to_string() + ), + // Validate order of operations is maintained + simple!( + col("modified") + .eq(lit("value")) + .and(col("value").eq(lit(1_i64))) + .or(col("modified") + .eq(lit("value2")) + .and(col("value").gt(lit(1_i64)))), + "modified = 'value' AND value = 1 OR modified = 'value2' AND value > 1".to_string() + ), + simple!( + col("modified") + .eq(lit("value")) + .or(col("value").eq(lit(1_i64))) + .and( + col("modified") + .eq(lit("value2")) + .or(col("value").gt(lit(1_i64))), + ), + "(modified = 'value' OR value = 1) AND (modified = 'value2' OR value > 1)" + .to_string() + ), + // Validate functions are correctly parsed + simple!( + substring(col("modified"), lit(0_i64), lit(4_i64)).eq(lit("2021")), + "substr(modified, 0, 4) = '2021'".to_string() + ), + simple!( + col("value") + .cast_to::( + &arrow_schema::DataType::Utf8, + &table + .state + .input_schema() + .unwrap() + .as_ref() + .to_owned() + .try_into() + .unwrap() + ) + .unwrap() + .eq(lit("1")), + "arrow_cast(value, 'Utf8') = '1'".to_string() + ), + ]; + + let session = SessionContext::new(); + + for test in tests { + let actual = fmt_expr_to_sql(&test.expr).unwrap(); + assert_eq!(test.expected, actual); + + let actual_expr = table + .state + .parse_predicate_expression(actual, &session.state()) + .unwrap(); + + match test.override_expected_expr { + None => assert_eq!(test.expr, actual_expr), + Some(expr) => assert_eq!(expr, actual_expr), + } + } + + let unsupported_types = vec![ + /* TODO: Determine proper way to display decimal values in an sql expression*/ + simple!( + col("money").gt(lit(ScalarValue::Decimal128(Some(100), 12, 2))), + "money > 0.1".to_string() + ), + simple!( + col("_timestamp").gt(lit(ScalarValue::TimestampMillisecond(Some(100), None))), + "".to_string() + ), + simple!( + col("_timestamp").gt(lit(ScalarValue::TimestampMillisecond( + Some(100), + Some("UTC".into()) + ))), + "".to_string() + ), + ]; + + for test in unsupported_types { + assert!(fmt_expr_to_sql(&test.expr).is_err()); + } + } +} diff --git a/rust/src/delta_datafusion.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs similarity index 71% rename from rust/src/delta_datafusion.rs rename to crates/deltalake-core/src/delta_datafusion/mod.rs index e69c7abbfe..7fbe362afc 100644 --- a/rust/src/delta_datafusion.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -8,7 +8,7 @@ //! //! async { //! let mut ctx = SessionContext::new(); -//! let table = deltalake::open_table("./tests/data/simple_table") +//! let table = deltalake_core::open_table("./tests/data/simple_table") //! .await //! .unwrap(); //! ctx.register_table("demo", Arc::new(table)).unwrap(); @@ -21,7 +21,7 @@ //! ``` use std::any::Any; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::convert::TryFrom; use std::fmt::{self, Debug}; use std::sync::Arc; @@ -32,12 +32,15 @@ use arrow::datatypes::DataType; use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, SchemaRef, TimeUnit}; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; -use arrow_array::StringArray; +use arrow_array::types::UInt16Type; +use arrow_array::{DictionaryArray, StringArray}; use arrow_schema::Field; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; use datafusion::datasource::file_format::{parquet::ParquetFormat, FileFormat}; -use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::datasource::physical_plan::{ + wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, +}; use datafusion::datasource::provider::TableProviderFactory; use datafusion::datasource::{listing::PartitionedFile, MemTable, TableProvider, TableType}; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; @@ -54,9 +57,7 @@ use datafusion::physical_plan::{ }; use datafusion_common::scalar::ScalarValue; use datafusion_common::tree_node::{TreeNode, TreeNodeVisitor, VisitRecursion}; -use datafusion_common::{ - Column, DFSchema, DataFusionError, Result as DataFusionResult, ToDFSchema, -}; +use datafusion_common::{Column, DataFusionError, Result as DataFusionResult, ToDFSchema}; use datafusion_expr::expr::{ScalarFunction, ScalarUDF}; use datafusion_expr::logical_plan::CreateExternalTable; use datafusion_expr::{col, Expr, Extension, LogicalPlan, TableProviderFilterPushDown, Volatility}; @@ -65,17 +66,20 @@ use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; use datafusion_proto::logical_plan::LogicalExtensionCodec; use datafusion_proto::physical_plan::PhysicalExtensionCodec; use object_store::ObjectMeta; +use serde::{Deserialize, Serialize}; use url::Url; -use crate::action::{self, Add}; -use crate::builder::ensure_table_uri; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{self, Add}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; +use crate::table::builder::ensure_table_uri; +use crate::table::state::DeltaTableState; use crate::{open_table, open_table_with_storage_options, DeltaTable, Invariant, SchemaDataType}; const PATH_COLUMN: &str = "__delta_rs_path"; +pub mod expr; + impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { match err { @@ -124,7 +128,7 @@ impl DeltaTableState { |acc, action| { let new_stats = action .get_stats() - .unwrap_or_else(|_| Some(action::Stats::default()))?; + .unwrap_or_else(|_| Some(protocol::Stats::default()))?; Some(Statistics { num_rows: acc .num_rows @@ -360,66 +364,298 @@ pub(crate) fn register_store(store: ObjectStoreRef, env: Arc) { env.register_object_store(url, store); } -/// Create a Parquet scan limited to a set of files -#[allow(clippy::too_many_arguments)] -pub(crate) async fn parquet_scan_from_actions( +pub(crate) fn logical_schema( snapshot: &DeltaTableState, + scan_config: &DeltaScanConfig, +) -> DeltaResult { + let input_schema = snapshot.input_schema()?; + let mut fields = Vec::new(); + for field in input_schema.fields.iter() { + fields.push(field.to_owned()); + } + + if let Some(file_column_name) = &scan_config.file_column_name { + fields.push(Arc::new(Field::new( + file_column_name, + arrow_schema::DataType::Utf8, + true, + ))); + } + + Ok(Arc::new(ArrowSchema::new(fields))) +} + +#[derive(Debug, Clone, Default)] +/// Used to specify if additonal metadata columns are exposed to the user +pub struct DeltaScanConfigBuilder { + /// Include the source path for each record. The name of this column is determine by `file_column_name` + include_file_column: bool, + /// Column name that contains the source path. + /// + /// If include_file_column is true and the name is None then it will be auto-generated + /// Otherwise the user provided name will be used + file_column_name: Option, +} + +impl DeltaScanConfigBuilder { + /// Construct a new instance of `DeltaScanConfigBuilder` + pub fn new() -> Self { + Self::default() + } + + /// Indicate that a column containing a records file path is included. + /// Column name is generated and can be determined once this Config is built + pub fn with_file_column(mut self, include: bool) -> Self { + self.include_file_column = include; + self.file_column_name = None; + self + } + + /// Indicate that a column containing a records file path is included and column name is user defined. + pub fn with_file_column_name(mut self, name: &S) -> Self { + self.file_column_name = Some(name.to_string()); + self.include_file_column = true; + self + } + + /// Build a DeltaScanConfig and ensure no column name conflicts occur during downstream processing + pub fn build(&self, snapshot: &DeltaTableState) -> DeltaResult { + let input_schema = snapshot.input_schema()?; + let mut file_column_name = None; + let mut column_names: HashSet<&String> = HashSet::new(); + for field in input_schema.fields.iter() { + column_names.insert(field.name()); + } + + if self.include_file_column { + match &self.file_column_name { + Some(name) => { + if column_names.contains(name) { + return Err(DeltaTableError::Generic(format!( + "Unable to add file path column since column with name {} exits", + name + ))); + } + + file_column_name = Some(name.to_owned()) + } + None => { + let prefix = PATH_COLUMN; + let mut idx = 0; + let mut name = prefix.to_owned(); + + while column_names.contains(&name) { + idx += 1; + name = format!("{}_{}", prefix, idx); + } + + file_column_name = Some(name); + } + } + } + + Ok(DeltaScanConfig { file_column_name }) + } +} + +#[derive(Debug, Clone, Default, Serialize, Deserialize)] +/// Include additonal metadata columns during a [`DeltaScan`] +pub struct DeltaScanConfig { + /// Include the source path for each record + pub file_column_name: Option, +} + +#[derive(Debug)] +pub(crate) struct DeltaScanBuilder<'a> { + snapshot: &'a DeltaTableState, object_store: ObjectStoreRef, - actions: &[Add], - schema: &ArrowSchema, - filters: Option>, - state: &SessionState, - projection: Option<&Vec>, + filter: Option, + state: &'a SessionState, + projection: Option<&'a Vec>, limit: Option, -) -> DataFusionResult> { - // TODO we group files together by their partition values. If the table is partitioned - // and partitions are somewhat evenly distributed, probably not the worst choice ... - // However we may want to do some additional balancing in case we are far off from the above. - let mut file_groups: HashMap, Vec> = HashMap::new(); - for action in actions { - let part = partitioned_file_from_action(action, schema); - file_groups - .entry(part.partition_values.clone()) - .or_default() - .push(part); - } - - let table_partition_cols = snapshot - .current_metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns - .clone(); - let file_schema = Arc::new(ArrowSchema::new( - schema - .fields() + files: Option<&'a [Add]>, + config: DeltaScanConfig, + schema: Option, +} + +impl<'a> DeltaScanBuilder<'a> { + pub fn new( + snapshot: &'a DeltaTableState, + object_store: ObjectStoreRef, + state: &'a SessionState, + ) -> Self { + DeltaScanBuilder { + snapshot, + object_store, + filter: None, + state, + files: None, + projection: None, + limit: None, + config: DeltaScanConfig::default(), + schema: None, + } + } + + pub fn with_filter(mut self, filter: Option) -> Self { + self.filter = filter; + self + } + + pub fn with_files(mut self, files: &'a [Add]) -> Self { + self.files = Some(files); + self + } + + pub fn with_projection(mut self, projection: Option<&'a Vec>) -> Self { + self.projection = projection; + self + } + + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + + pub fn with_scan_config(mut self, config: DeltaScanConfig) -> Self { + self.config = config; + self + } + + pub fn with_schema(mut self, schema: SchemaRef) -> Self { + self.schema = Some(schema); + self + } + + pub async fn build(self) -> DeltaResult { + let config = self.config; + let schema = match self.schema { + Some(schema) => schema, + None => { + self.snapshot + .physical_arrow_schema(self.object_store.clone()) + .await? + } + }; + let logical_schema = logical_schema(self.snapshot, &config)?; + + let logical_schema = if let Some(used_columns) = self.projection { + let mut fields = vec![]; + for idx in used_columns { + fields.push(logical_schema.field(*idx).to_owned()); + } + Arc::new(ArrowSchema::new(fields)) + } else { + logical_schema + }; + + let logical_filter = self + .filter + .map(|expr| logical_expr_to_physical_expr(&expr, &logical_schema)); + + // Perform Pruning of files to scan + let files = match self.files { + Some(files) => files.to_owned(), + None => { + if let Some(predicate) = &logical_filter { + let pruning_predicate = + PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; + let files_to_prune = pruning_predicate.prune(self.snapshot)?; + self.snapshot + .files() + .iter() + .zip(files_to_prune.into_iter()) + .filter_map( + |(action, keep)| { + if keep { + Some(action.to_owned()) + } else { + None + } + }, + ) + .collect() + } else { + self.snapshot.files().to_owned() + } + } + }; + + // TODO we group files together by their partition values. If the table is partitioned + // and partitions are somewhat evenly distributed, probably not the worst choice ... + // However we may want to do some additional balancing in case we are far off from the above. + let mut file_groups: HashMap, Vec> = HashMap::new(); + + let table_partition_cols = &self + .snapshot + .current_metadata() + .ok_or(DeltaTableError::NoMetadata)? + .partition_columns; + + for action in files.iter() { + let mut part = partitioned_file_from_action(action, table_partition_cols, &schema); + + if config.file_column_name.is_some() { + part.partition_values + .push(wrap_partition_value_in_dict(ScalarValue::Utf8(Some( + action.path.clone(), + )))); + } + + file_groups + .entry(part.partition_values.clone()) + .or_default() + .push(part); + } + + let file_schema = Arc::new(ArrowSchema::new( + schema + .fields() + .iter() + .filter(|f| !table_partition_cols.contains(f.name())) + .cloned() + .collect::>(), + )); + + let mut table_partition_cols = table_partition_cols .iter() - .filter(|f| !table_partition_cols.contains(f.name())) - .cloned() - .collect::>(), - )); + .map(|c| Ok((c.to_owned(), schema.field_with_name(c)?.data_type().clone()))) + .collect::, ArrowError>>()?; - let table_partition_cols = table_partition_cols - .iter() - .map(|c| Ok((c.to_owned(), schema.field_with_name(c)?.data_type().clone()))) - .collect::, ArrowError>>()?; + if let Some(file_column_name) = &config.file_column_name { + table_partition_cols.push(( + file_column_name.clone(), + wrap_partition_type_in_dict(DataType::Utf8), + )); + } - ParquetFormat::new() - .create_physical_plan( - state, - FileScanConfig { - object_store_url: object_store.object_store_url(), - file_schema, - file_groups: file_groups.into_values().collect(), - statistics: snapshot.datafusion_table_statistics(), - projection: projection.cloned(), - limit, - table_partition_cols, - output_ordering: vec![], - infinite_source: false, - }, - filters.as_ref(), - ) - .await + let scan = ParquetFormat::new() + .create_physical_plan( + self.state, + FileScanConfig { + object_store_url: self.object_store.object_store_url(), + file_schema, + file_groups: file_groups.into_values().collect(), + statistics: self.snapshot.datafusion_table_statistics(), + projection: self.projection.cloned(), + limit: self.limit, + table_partition_cols, + output_ordering: vec![], + infinite_source: false, + }, + logical_filter.as_ref(), + ) + .await?; + + Ok(DeltaScan { + table_uri: ensure_table_uri(self.object_store.root_uri())? + .as_str() + .into(), + parquet_scan: scan, + config, + logical_schema, + }) + } } #[async_trait] @@ -451,53 +687,96 @@ impl TableProvider for DeltaTable { filters: &[Expr], limit: Option, ) -> DataFusionResult> { - let schema = self - .state - .physical_arrow_schema(self.object_store()) + register_store(self.object_store(), session.runtime_env().clone()); + let filter_expr = conjunction(filters.iter().cloned()); + + let scan = DeltaScanBuilder::new(&self.state, self.object_store(), session) + .with_projection(projection) + .with_limit(limit) + .with_filter(filter_expr) + .build() .await?; - register_store(self.object_store(), session.runtime_env().clone()); + Ok(Arc::new(scan)) + } - let filter_expr = conjunction(filters.iter().cloned()) - .map(|expr| logical_expr_to_physical_expr(&expr, &schema)); + fn supports_filter_pushdown( + &self, + _filter: &Expr, + ) -> DataFusionResult { + Ok(TableProviderFilterPushDown::Inexact) + } - let actions = if let Some(predicate) = &filter_expr { - let pruning_predicate = PruningPredicate::try_new(predicate.clone(), schema.clone())?; - let files_to_prune = pruning_predicate.prune(&self.state)?; - self.get_state() - .files() - .iter() - .zip(files_to_prune) - .filter_map( - |(action, keep)| { - if keep { - Some(action.to_owned()) - } else { - None - } - }, - ) - .collect() - } else { - self.get_state().files().to_owned() - }; + fn statistics(&self) -> Option { + Some(self.state.datafusion_table_statistics()) + } +} - let parquet_scan = parquet_scan_from_actions( - &self.state, - self.object_store(), - &actions, - &schema, - filter_expr, - session, - projection, - limit, - ) - .await?; +/// A Delta table provider that enables additonal metadata columns to be included during the scan +pub struct DeltaTableProvider { + snapshot: DeltaTableState, + store: ObjectStoreRef, + config: DeltaScanConfig, + schema: Arc, +} + +impl DeltaTableProvider { + /// Build a DeltaTableProvider + pub fn try_new( + snapshot: DeltaTableState, + store: ObjectStoreRef, + config: DeltaScanConfig, + ) -> DeltaResult { + Ok(DeltaTableProvider { + schema: logical_schema(&snapshot, &config)?, + snapshot, + store, + config, + }) + } +} - Ok(Arc::new(DeltaScan { - table_uri: ensure_table_uri(self.table_uri())?.as_str().into(), - parquet_scan, - })) +#[async_trait] +impl TableProvider for DeltaTableProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> Arc { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + fn get_table_definition(&self) -> Option<&str> { + None + } + + fn get_logical_plan(&self) -> Option<&LogicalPlan> { + None + } + + async fn scan( + &self, + session: &SessionState, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + ) -> DataFusionResult> { + register_store(self.store.clone(), session.runtime_env().clone()); + let filter_expr = conjunction(filters.iter().cloned()); + + let scan = DeltaScanBuilder::new(&self.snapshot, self.store.clone(), session) + .with_projection(projection) + .with_limit(limit) + .with_filter(filter_expr) + .with_scan_config(self.config.clone()) + .build() + .await?; + + Ok(Arc::new(scan)) } fn supports_filter_pushdown( @@ -508,7 +787,7 @@ impl TableProvider for DeltaTable { } fn statistics(&self) -> Option { - Some(self.state.datafusion_table_statistics()) + Some(self.snapshot.datafusion_table_statistics()) } } @@ -518,8 +797,19 @@ impl TableProvider for DeltaTable { pub struct DeltaScan { /// The URL of the ObjectStore root pub table_uri: String, + /// Column that contains an index that maps to the original metadata Add + pub config: DeltaScanConfig, /// The parquet scan to wrap pub parquet_scan: Arc, + /// The schema of the table to be used when evaluating expressions + pub logical_schema: Arc, +} + +#[derive(Debug, Serialize, Deserialize)] +struct DeltaScanWire { + pub table_uri: String, + pub config: DeltaScanConfig, + pub logical_schema: Arc, } impl DisplayAs for DeltaScan { @@ -631,21 +921,31 @@ pub(crate) fn get_null_of_arrow_type(t: &ArrowDataType) -> DeltaResult PartitionedFile { - let partition_values = schema - .fields() + let partition_values = partition_columns .iter() - .filter_map(|f| { - action.partition_values.get(f.name()).map(|val| match val { - Some(value) => to_correct_scalar_value( - &serde_json::Value::String(value.to_string()), - f.data_type(), - ) - .unwrap_or(ScalarValue::Null), - None => get_null_of_arrow_type(f.data_type()).unwrap_or(ScalarValue::Null), - }) + .map(|part| { + action + .partition_values + .get(part) + .map(|val| { + schema + .field_with_name(part) + .map(|field| match val { + Some(value) => to_correct_scalar_value( + &serde_json::Value::String(value.to_string()), + field.data_type(), + ) + .unwrap_or(ScalarValue::Null), + None => get_null_of_arrow_type(field.data_type()) + .unwrap_or(ScalarValue::Null), + }) + .unwrap_or(ScalarValue::Null) + }) + .unwrap_or(ScalarValue::Null) }) .collect::>(); @@ -920,11 +1220,13 @@ impl PhysicalExtensionCodec for DeltaPhysicalCodec { inputs: &[Arc], _registry: &dyn FunctionRegistry, ) -> Result, DataFusionError> { - let table_uri: String = serde_json::from_reader(buf) + let wire: DeltaScanWire = serde_json::from_reader(buf) .map_err(|_| DataFusionError::Internal("Unable to decode DeltaScan".to_string()))?; let delta_scan = DeltaScan { - table_uri, + table_uri: wire.table_uri, parquet_scan: (*inputs)[0].clone(), + config: wire.config, + logical_schema: wire.logical_schema, }; Ok(Arc::new(delta_scan)) } @@ -938,7 +1240,13 @@ impl PhysicalExtensionCodec for DeltaPhysicalCodec { .as_any() .downcast_ref::() .ok_or_else(|| DataFusionError::Internal("Not a delta scan!".to_string()))?; - serde_json::to_writer(buf, delta_scan.table_uri.as_str()) + + let wire = DeltaScanWire { + table_uri: delta_scan.table_uri.to_owned(), + config: delta_scan.config.clone(), + logical_schema: delta_scan.logical_schema.clone(), + }; + serde_json::to_writer(buf, &wire) .map_err(|_| DataFusionError::Internal("Unable to encode delta scan!".to_string()))?; Ok(()) } @@ -1099,27 +1407,44 @@ pub struct FindFiles { fn join_batches_with_add_actions( batches: Vec, mut actions: HashMap, + path_column: &str, + dict_array: bool, ) -> DeltaResult> { // Given RecordBatches that contains `__delta_rs_path` perform a hash join // with actions to obtain original 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)) - })? - .as_any() - .downcast_ref::() - .ok_or(DeltaTableError::Generic(format!( - "Unable to downcast column {}", - PATH_COLUMN - )))?; - for path in array { + 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()) + }; + + for path in iter { let path = path.ok_or(DeltaTableError::Generic(format!( "{} cannot be null", - PATH_COLUMN + path_column )))?; match actions.remove(path) { @@ -1139,89 +1464,43 @@ fn join_batches_with_add_actions( pub(crate) async fn find_files_scan<'a>( snapshot: &DeltaTableState, store: ObjectStoreRef, - schema: Arc, - file_schema: Arc, - candidates: Vec<&'a Add>, state: &SessionState, - expression: &Expr, + expression: Expr, ) -> DeltaResult> { - let mut candidate_map: HashMap = HashMap::new(); - - let table_partition_cols = snapshot - .current_metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns - .clone(); - - let mut file_groups: HashMap, Vec> = HashMap::new(); - for action in candidates { - let mut part = partitioned_file_from_action(action, &schema); - part.partition_values - .push(ScalarValue::Utf8(Some(action.path.clone()))); - - file_groups - .entry(part.partition_values.clone()) - .or_default() - .push(part); - - candidate_map.insert(action.path.to_owned(), action.to_owned()); - } - - let mut table_partition_cols = table_partition_cols + let candidate_map: HashMap = snapshot + .files() .iter() - .map(|c| Ok((c.to_owned(), schema.field_with_name(c)?.data_type().clone()))) - .collect::, ArrowError>>()?; - // Append a column called __delta_rs_path to track the file path - table_partition_cols.push((PATH_COLUMN.to_owned(), DataType::Utf8)); + .map(|add| (add.path.clone(), add.to_owned())) + .collect(); - let input_schema = snapshot.input_schema()?; - - let mut fields = Vec::new(); - for field in input_schema.fields.iter() { - fields.push(field.to_owned()); + let scan_config = DeltaScanConfigBuilder { + include_file_column: true, + ..Default::default() } - fields.push(Arc::new(Field::new( - PATH_COLUMN, - arrow_schema::DataType::Boolean, - true, - ))); - let input_schema = Arc::new(ArrowSchema::new(fields)); + .build(snapshot)?; + + let logical_schema = logical_schema(snapshot, &scan_config)?; // Identify which columns we need to project let mut used_columns = expression .to_columns()? .into_iter() - .map(|column| input_schema.index_of(&column.name)) - .collect::, ArrowError>>() - .unwrap(); + .map(|column| logical_schema.index_of(&column.name)) + .collect::, ArrowError>>()?; // Add path column - used_columns.push(input_schema.index_of(PATH_COLUMN)?); + used_columns.push(logical_schema.index_of(scan_config.file_column_name.as_ref().unwrap())?); - // Project the logical schema so column indicies align between the parquet scan and the expression - let mut fields = vec![]; - for idx in &used_columns { - fields.push(input_schema.field(*idx).to_owned()); - } - let input_schema = Arc::new(ArrowSchema::new(fields)); - let input_dfschema = input_schema.as_ref().clone().try_into()?; - - let parquet_scan = ParquetFormat::new() - .create_physical_plan( - state, - FileScanConfig { - object_store_url: store.object_store_url(), - file_schema, - file_groups: file_groups.into_values().collect(), - statistics: snapshot.datafusion_table_statistics(), - projection: Some(used_columns), - limit: None, - table_partition_cols, - infinite_source: false, - output_ordering: vec![], - }, - None, - ) + let scan = DeltaScanBuilder::new(snapshot, store.clone(), state) + .with_filter(Some(expression.clone())) + .with_projection(Some(&used_columns)) + .with_scan_config(scan_config) + .build() .await?; + let scan = Arc::new(scan); + + let config = &scan.config; + let input_schema = scan.logical_schema.as_ref().to_owned(); + let input_dfschema = input_schema.clone().try_into()?; let predicate_expr = create_physical_expr( &Expr::IsTrue(Box::new(expression.clone())), @@ -1231,13 +1510,18 @@ pub(crate) async fn find_files_scan<'a>( )?; let filter: Arc = - Arc::new(FilterExec::try_new(predicate_expr, parquet_scan.clone())?); + Arc::new(FilterExec::try_new(predicate_expr, scan.clone())?); let limit: Arc = Arc::new(LocalLimitExec::new(filter, 1)); let task_ctx = Arc::new(TaskContext::from(state)); let path_batches = datafusion::physical_plan::collect(limit, task_ctx).await?; - join_batches_with_add_actions(path_batches, candidate_map) + join_batches_with_add_actions( + path_batches, + candidate_map, + config.file_column_name.as_ref().unwrap(), + true, + ) } pub(crate) async fn scan_memory_table( @@ -1291,42 +1575,22 @@ pub(crate) async fn scan_memory_table( .map(|action| (action.path.clone(), action)) .collect::>(); - join_batches_with_add_actions(batches, map) + join_batches_with_add_actions(batches, map, PATH_COLUMN, false) } /// Finds files in a snapshot that match the provided predicate. pub async fn find_files<'a>( snapshot: &DeltaTableState, object_store: ObjectStoreRef, - schema: Arc, state: &SessionState, predicate: Option, ) -> DeltaResult { let current_metadata = snapshot .current_metadata() .ok_or(DeltaTableError::NoMetadata)?; - let table_partition_cols = current_metadata.partition_columns.clone(); match &predicate { Some(predicate) => { - let file_schema = Arc::new(ArrowSchema::new( - schema - .fields() - .iter() - .filter(|f| !table_partition_cols.contains(f.name())) - .cloned() - .collect::>(), - )); - - let input_schema = snapshot.input_schema()?; - let input_dfschema: DFSchema = input_schema.clone().as_ref().clone().try_into()?; - let expr = create_physical_expr( - predicate, - &input_dfschema, - &input_schema, - state.execution_props(), - )?; - // Validate the Predicate and determine if it only contains partition columns let mut expr_properties = FindFilesExprProperties { partition_only: true, @@ -1344,26 +1608,9 @@ pub async fn find_files<'a>( partition_scan: true, }) } else { - let pruning_predicate = PruningPredicate::try_new(expr, schema.clone())?; - let files_to_prune = pruning_predicate.prune(snapshot)?; - let files: Vec<&Add> = snapshot - .files() - .iter() - .zip(files_to_prune.into_iter()) - .filter_map(|(action, keep)| if keep { Some(action) } else { None }) - .collect(); - - // Create a new delta scan plan with only files that have a record - let candidates = find_files_scan( - snapshot, - object_store.clone(), - schema.clone(), - file_schema.clone(), - files, - state, - predicate, - ) - .await?; + let candidates = + find_files_scan(snapshot, object_store.clone(), state, predicate.to_owned()) + .await?; Ok(FindFiles { candidates, @@ -1380,9 +1627,11 @@ pub async fn find_files<'a>( #[cfg(test)] mod tests { + use crate::writer::test_utils::get_delta_schema; use arrow::array::StructArray; use arrow::datatypes::{DataType, Field, Schema}; use chrono::{TimeZone, Utc}; + use datafusion::assert_batches_sorted_eq; use datafusion::physical_plan::empty::EmptyExec; use datafusion_proto::physical_plan::AsExecutionPlan; use datafusion_proto::protobuf; @@ -1541,7 +1790,7 @@ mod tests { let mut partition_values = std::collections::HashMap::new(); partition_values.insert("month".to_string(), Some("1".to_string())); partition_values.insert("year".to_string(), Some("2015".to_string())); - let action = action::Add { + let action = protocol::Add { path: "year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string(), size: 10644, partition_values, @@ -1558,7 +1807,8 @@ mod tests { Field::new("month", ArrowDataType::Int64, true), ]); - let file = partitioned_file_from_action(&action, &schema); + let part_columns = vec!["year".to_string(), "month".to_string()]; + let file = partitioned_file_from_action(&action, &part_columns, &schema); let ref_file = PartitionedFile { object_meta: object_store::ObjectMeta { location: Path::from("year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string()), @@ -1648,7 +1898,9 @@ 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)), + parquet_scan: Arc::from(EmptyExec::new(false, schema.clone())), + config: DeltaScanConfig::default(), + logical_schema: schema.clone(), }); let proto: protobuf::PhysicalPlanNode = protobuf::PhysicalPlanNode::try_from_physical_plan(exec_plan.clone(), &codec) @@ -1660,4 +1912,92 @@ mod tests { .expect("from proto"); assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); } + + #[tokio::test] + async fn delta_table_provider_with_config() { + let table = crate::open_table("tests/data/delta-2.2.0-partitioned-types") + .await + .unwrap(); + let config = DeltaScanConfigBuilder::new() + .with_file_column_name(&"file_source") + .build(&table.state) + .unwrap(); + + let provider = DeltaTableProvider::try_new(table.state, table.storage, config).unwrap(); + let ctx = SessionContext::new(); + ctx.register_table("test", Arc::new(provider)).unwrap(); + + let df = ctx.sql("select * from test").await.unwrap(); + let actual = df.collect().await.unwrap(); + let expected = vec! [ + "+----+----+----+-------------------------------------------------------------------------------+", + "| c3 | c1 | c2 | file_source |", + "+----+----+----+-------------------------------------------------------------------------------+", + "| 4 | 6 | a | c1=6/c2=a/part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet |", + "| 5 | 4 | c | c1=4/c2=c/part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet |", + "| 6 | 5 | b | c1=5/c2=b/part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet |", + "+----+----+----+-------------------------------------------------------------------------------+", + ]; + assert_batches_sorted_eq!(&expected, &actual); + } + + #[tokio::test] + async fn delta_scan_mixed_partition_order() { + // Tests issue (1787) where partition columns were incorrect when they + // have a different order in the metadata and table schema + 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 table = crate::DeltaOps::new_in_memory() + .create() + .with_columns(get_delta_schema().get_fields().clone()) + .with_partition_columns(["modified", "id"]) + .await + .unwrap(); + assert_eq!(table.version(), 0); + + 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(table) + .write(vec![batch.clone()]) + .with_save_mode(crate::protocol::SaveMode::Append) + .await + .unwrap(); + + let config = DeltaScanConfigBuilder::new().build(&table.state).unwrap(); + + let provider = DeltaTableProvider::try_new(table.state, table.storage, config).unwrap(); + let ctx = SessionContext::new(); + ctx.register_table("test", Arc::new(provider)).unwrap(); + + let df = ctx.sql("select * from test").await.unwrap(); + let actual = df.collect().await.unwrap(); + let expected = vec![ + "+-------+------------+----+", + "| value | modified | id |", + "+-------+------------+----+", + "| 1 | 2021-02-01 | A |", + "| 10 | 2021-02-01 | B |", + "| 100 | 2021-02-02 | D |", + "| 20 | 2021-02-02 | C |", + "+-------+------------+----+", + ]; + assert_batches_sorted_eq!(&expected, &actual); + } } diff --git a/rust/src/errors.rs b/crates/deltalake-core/src/errors.rs similarity index 99% rename from rust/src/errors.rs rename to crates/deltalake-core/src/errors.rs index fed0e823f8..24989b2814 100644 --- a/rust/src/errors.rs +++ b/crates/deltalake-core/src/errors.rs @@ -1,8 +1,8 @@ //! Exceptions for the deltalake crate use object_store::Error as ObjectStoreError; -use crate::action::ProtocolError; use crate::operations::transaction::TransactionError; +use crate::protocol::ProtocolError; /// A result returned by delta-rs pub type DeltaResult = Result; diff --git a/rust/src/lib.rs b/crates/deltalake-core/src/lib.rs similarity index 83% rename from rust/src/lib.rs rename to crates/deltalake-core/src/lib.rs index 2d03651cb9..fa7f65963f 100644 --- a/rust/src/lib.rs +++ b/crates/deltalake-core/src/lib.rs @@ -6,7 +6,7 @@ //! //! ```rust //! async { -//! let table = deltalake::open_table("./tests/data/simple_table").await.unwrap(); +//! let table = deltalake_core::open_table("./tests/data/simple_table").await.unwrap(); //! let files = table.get_files(); //! }; //! ``` @@ -15,10 +15,10 @@ //! //! ```rust //! async { -//! let table = deltalake::open_table_with_version("./tests/data/simple_table", 0).await.unwrap(); -//! let files = table.get_files_by_partitions(&[deltalake::PartitionFilter { -//! key: "month", -//! value: deltalake::PartitionValue::Equal("12"), +//! let table = deltalake_core::open_table_with_version("./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()), //! }]); //! }; //! ``` @@ -27,7 +27,7 @@ //! //! ```rust //! async { -//! let table = deltalake::open_table_with_ds( +//! let table = deltalake_core::open_table_with_ds( //! "./tests/data/simple_table", //! "2020-05-02T23:47:31-07:00", //! ).await.unwrap(); @@ -56,7 +56,7 @@ //! //! async { //! let mut ctx = SessionContext::new(); -//! let table = deltalake::open_table("./tests/data/simple_table") +//! let table = deltalake_core::open_table("./tests/data/simple_table") //! .await //! .unwrap(); //! ctx.register_table("demo", Arc::new(table)).unwrap(); @@ -82,42 +82,34 @@ compile_error!( "Features s3 and s3-native-tls are mutually exclusive and cannot be enabled together" ); -pub mod action; -pub mod builder; pub mod data_catalog; -pub mod delta; -pub mod delta_config; pub mod errors; pub mod operations; -pub mod partitions; +pub mod protocol; pub mod schema; pub mod storage; -pub mod table_state; -pub mod time_utils; +pub mod table; -#[cfg(feature = "arrow")] -pub mod table_state_arrow; - -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod delta_arrow; #[cfg(feature = "datafusion")] pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; -pub use self::builder::*; +use std::collections::HashMap; + pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; -pub use self::delta::*; -pub use self::delta_config::*; -pub use self::partitions::*; +pub use self::errors::*; +pub use self::schema::partitions::*; pub use self::schema::*; -pub use errors::*; +pub use self::table::builder::{ + DeltaTableBuilder, DeltaTableConfig, DeltaTableLoadOptions, DeltaVersion, +}; +pub use self::table::config::DeltaConfigKey; +pub use self::table::DeltaTable; pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; pub use operations::DeltaOps; // convenience exports for consumers to avoid aligning crate versions -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub use action::checkpoints; #[cfg(feature = "arrow")] pub use arrow; #[cfg(feature = "datafusion")] @@ -126,15 +118,70 @@ pub use datafusion; 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. +pub async fn open_table(table_uri: impl AsRef) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri).load().await?; + Ok(table) +} + +/// Same as `open_table`, but also accepts storage options to aid in building the table for a deduced +/// `StorageService`. +pub async fn open_table_with_storage_options( + table_uri: impl AsRef, + storage_options: HashMap, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_storage_options(storage_options) + .load() + .await?; + Ok(table) +} + +/// Creates a DeltaTable from the given path and loads it with the metadata from the given version. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table_with_version( + table_uri: impl AsRef, + version: i64, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_version(version) + .load() + .await?; + Ok(table) +} + +/// Creates a DeltaTable from the given path. +/// Loads metadata from the version appropriate based on the given ISO-8601/RFC-3339 timestamp. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table_with_ds( + table_uri: impl AsRef, + ds: impl AsRef, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_datestring(ds)? + .load() + .await?; + Ok(table) +} + +/// Returns rust crate version, can be use used in language bindings to expose Rust core version +pub fn crate_version() -> &'static str { + env!("CARGO_PKG_VERSION") +} + #[cfg(test)] mod tests { use super::*; + use crate::table::PeekCommit; use std::collections::HashMap; #[tokio::test] @@ -153,7 +200,7 @@ mod tests { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 4); - assert!(tombstones.contains(&crate::action::Remove { + assert!(tombstones.contains(&crate::protocol::Remove { path: "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1564524298213), data_change: false, @@ -255,7 +302,7 @@ mod tests { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 1); - assert!(tombstones.contains(&crate::action::Remove { + assert!(tombstones.contains(&crate::protocol::Remove { path: "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1615043776198), data_change: true, @@ -301,12 +348,12 @@ mod tests { let filters = vec![ crate::PartitionFilter { - key: "month", - value: crate::PartitionValue::Equal("2"), + key: "month".to_string(), + value: crate::PartitionValue::Equal("2".to_string()), }, crate::PartitionFilter { - key: "year", - value: crate::PartitionValue::Equal("2020"), + key: "year".to_string(), + value: crate::PartitionValue::Equal("2020".to_string()), }, ]; @@ -336,8 +383,8 @@ mod tests { ); let filters = vec![crate::PartitionFilter { - key: "month", - value: crate::PartitionValue::NotEqual("2"), + key: "month".to_string(), + value: crate::PartitionValue::NotEqual("2".to_string()), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -350,8 +397,8 @@ mod tests { ); let filters = vec![crate::PartitionFilter { - key: "month", - value: crate::PartitionValue::In(vec!["2", "12"]), + key: "month".to_string(), + value: crate::PartitionValue::In(vec!["2".to_string(), "12".to_string()]), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -364,8 +411,8 @@ mod tests { ); let filters = vec![crate::PartitionFilter { - key: "month", - value: crate::PartitionValue::NotIn(vec!["2", "12"]), + key: "month".to_string(), + value: crate::PartitionValue::NotIn(vec!["2".to_string(), "12".to_string()]), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -383,8 +430,8 @@ mod tests { .unwrap(); let filters = vec![crate::PartitionFilter { - key: "k", - value: crate::PartitionValue::Equal("A"), + key: "k".to_string(), + value: crate::PartitionValue::Equal("A".to_string()), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -394,8 +441,8 @@ mod tests { ); let filters = vec![crate::PartitionFilter { - key: "k", - value: crate::PartitionValue::Equal(""), + key: "k".to_string(), + value: crate::PartitionValue::Equal("".to_string()), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -426,8 +473,8 @@ mod tests { ); let filters = vec![crate::PartitionFilter { - key: "x", - value: crate::PartitionValue::Equal("A/A"), + key: "x".to_string(), + value: crate::PartitionValue::Equal("A/A".to_string()), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -445,8 +492,8 @@ mod tests { .unwrap(); let filters = vec![crate::PartitionFilter { - key: "x", - value: crate::PartitionValue::LessThanOrEqual("9"), + key: "x".to_string(), + value: crate::PartitionValue::LessThanOrEqual("9".to_string()), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -456,8 +503,8 @@ mod tests { ); let filters = vec![crate::PartitionFilter { - key: "y", - value: crate::PartitionValue::LessThan("10.0"), + key: "y".to_string(), + value: crate::PartitionValue::LessThan("10.0".to_string()), }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), @@ -475,7 +522,10 @@ mod tests { 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::action::Stats { + fn get_stats_for_file( + table: &crate::DeltaTable, + file_name: &str, + ) -> crate::protocol::Stats { table .get_file_uris() .zip(table.get_stats()) diff --git a/rust/src/operations/create.rs b/crates/deltalake-core/src/operations/create.rs similarity index 92% rename from rust/src/operations/create.rs rename to crates/deltalake-core/src/operations/create.rs index 697ab3ef1d..8a78f2266b 100644 --- a/rust/src/operations/create.rs +++ b/crates/deltalake-core/src/operations/create.rs @@ -9,13 +9,14 @@ use serde_json::{Map, Value}; use super::transaction::commit; use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; -use crate::action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; -use crate::builder::ensure_table_uri; -use crate::delta_config::DeltaConfigKey; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; use crate::storage::DeltaObjectStore; -use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData}; +use crate::table::builder::ensure_table_uri; +use crate::table::config::DeltaConfigKey; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder}; #[derive(thiserror::Error, Debug)] enum CreateError { @@ -148,7 +149,7 @@ impl CreateBuilder { /// /// Options may be passed in the HashMap or set as environment variables. /// - /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. + /// [crate::table::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. /// /// If an object store is also passed using `with_object_store()` these options will be ignored. @@ -249,6 +250,8 @@ impl CreateBuilder { .unwrap_or_else(|| Protocol { min_reader_version: MAX_SUPPORTED_READER_VERSION, min_writer_version: MAX_SUPPORTED_WRITER_VERSION, + writer_features: None, + reader_features: None, }); let metadata = DeltaTableMetaData::new( @@ -291,7 +294,7 @@ impl std::future::IntoFuture for CreateBuilder { Box::pin(async move { let mode = this.mode.clone(); let (mut table, actions, operation) = this.into_table_and_actions()?; - if table.object_store().is_delta_table_location().await? { + let table_state = if table.object_store().is_delta_table_location().await? { match mode { SaveMode::ErrorIfExists => return Err(CreateError::TableAlreadyExists.into()), SaveMode::Append => return Err(CreateError::AppendNotAllowed.into()), @@ -300,15 +303,19 @@ impl std::future::IntoFuture for CreateBuilder { return Ok(table); } SaveMode::Overwrite => { - todo!("Overwriting on create not yet implemented. Use 'write' operation instead.") + table.load().await?; + &table.state } } - } + } else { + &table.state + }; + let version = commit( table.object_store().as_ref(), &actions, operation, - &table.state, + table_state, None, ) .await?; @@ -322,8 +329,8 @@ impl std::future::IntoFuture for CreateBuilder { #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; - use crate::delta_config::DeltaConfigKey; use crate::operations::DeltaOps; + use crate::table::config::DeltaConfigKey; use crate::writer::test_utils::get_delta_schema; use tempdir::TempDir; @@ -394,6 +401,8 @@ mod tests { let protocol = Protocol { min_reader_version: 0, min_writer_version: 0, + writer_features: None, + reader_features: None, }; let table = CreateBuilder::new() .with_location("memory://") @@ -455,12 +464,12 @@ mod tests { assert_eq!(table.get_metadata().unwrap().id, first_id); // Check table is overwritten - // let table = CreateBuilder::new() - // .with_object_store(object_store.clone()) - // .with_columns(schema.get_fields().clone()) - // .with_save_mode(SaveMode::Overwrite) - // .await - // .unwrap(); - // assert_ne!(table.get_metadata().unwrap().id, first_id) + let table = CreateBuilder::new() + .with_object_store(object_store.clone()) + .with_columns(schema.get_fields().clone()) + .with_save_mode(SaveMode::Overwrite) + .await + .unwrap(); + assert_ne!(table.get_metadata().unwrap().id, first_id) } } diff --git a/rust/src/operations/delete.rs b/crates/deltalake-core/src/operations/delete.rs similarity index 91% rename from rust/src/operations/delete.rs rename to crates/deltalake-core/src/operations/delete.rs index 39b98cce24..913658f279 100644 --- a/rust/src/operations/delete.rs +++ b/crates/deltalake-core/src/operations/delete.rs @@ -20,7 +20,8 @@ use std::sync::Arc; use std::time::{Instant, SystemTime, UNIX_EPOCH}; -use crate::action::{Action, Add, Remove}; +use crate::delta_datafusion::expr::fmt_expr_to_sql; +use crate::protocol::{Action, Add, Remove}; use datafusion::execution::context::{SessionContext, SessionState}; use datafusion::physical_expr::create_physical_expr; use datafusion::physical_plan::filter::FilterExec; @@ -30,16 +31,17 @@ use datafusion_common::scalar::ScalarValue; use datafusion_common::DFSchema; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; +use serde::Serialize; use serde_json::Map; use serde_json::Value; -use crate::action::DeltaOperation; -use crate::delta_datafusion::{find_files, parquet_scan_from_actions, register_store}; +use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; use crate::operations::write::write_execution_plan; +use crate::protocol::DeltaOperation; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; use super::datafusion_utils::Expression; @@ -61,7 +63,7 @@ pub struct DeleteBuilder { app_metadata: Option>, } -#[derive(Default, Debug)] +#[derive(Default, Debug, Serialize)] /// Metrics for the Delete Operation pub struct DeleteMetrics { /// Number of files added @@ -114,7 +116,7 @@ impl DeleteBuilder { self } - /// Writer properties passed to parquet writer for when fiiles are rewritten + /// Writer properties passed to parquet writer for when files are rewritten pub fn with_writer_properties(mut self, writer_properties: WriterProperties) -> Self { self.writer_properties = Some(writer_properties); self @@ -133,7 +135,6 @@ async fn excute_non_empty_expr( // 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 schema = snapshot.arrow_schema()?; let input_schema = snapshot.input_schema()?; let input_dfschema: DFSchema = input_schema.clone().as_ref().clone().try_into()?; @@ -143,17 +144,11 @@ async fn excute_non_empty_expr( .partition_columns .clone(); - let parquet_scan = parquet_scan_from_actions( - snapshot, - object_store.clone(), - rewrite, - &schema, - None, - state, - None, - None, - ) - .await?; + let scan = DeltaScanBuilder::new(snapshot, object_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())))); @@ -165,7 +160,7 @@ async fn excute_non_empty_expr( state.execution_props(), )?; let filter: Arc = - Arc::new(FilterExec::try_new(predicate_expr, parquet_scan.clone())?); + Arc::new(FilterExec::try_new(predicate_expr, scan.clone())?); let add_actions = write_execution_plan( snapshot, @@ -180,7 +175,7 @@ async fn excute_non_empty_expr( ) .await?; - let read_records = parquet_scan.metrics().and_then(|m| m.output_rows()); + let read_records = scan.parquet_scan.metrics().and_then(|m| m.output_rows()); let filter_records = filter.metrics().and_then(|m| m.output_rows()); metrics.num_copied_rows = filter_records; metrics.num_deleted_rows = read_records @@ -200,17 +195,9 @@ async fn execute( ) -> DeltaResult<((Vec, i64), DeleteMetrics)> { let exec_start = Instant::now(); let mut metrics = DeleteMetrics::default(); - let schema = snapshot.arrow_schema()?; let scan_start = Instant::now(); - let candidates = find_files( - snapshot, - object_store.clone(), - schema.clone(), - &state, - predicate.clone(), - ) - .await?; + let candidates = find_files(snapshot, object_store.clone(), &state, predicate.clone()).await?; metrics.scan_time_ms = Instant::now().duration_since(scan_start).as_micros(); let predicate = predicate.unwrap_or(Expr::Literal(ScalarValue::Boolean(Some(true)))); @@ -262,7 +249,7 @@ async fn execute( // Do not make a commit when there are zero updates to the state if !actions.is_empty() { let operation = DeltaOperation::Delete { - predicate: Some(predicate.canonical_name()), + predicate: Some(fmt_expr_to_sql(&predicate)?), }; version = commit( object_store.as_ref(), @@ -297,7 +284,9 @@ impl std::future::IntoFuture for DeleteBuilder { let predicate = match this.predicate { Some(predicate) => match predicate { Expression::DataFusion(expr) => Some(expr), - Expression::String(s) => Some(this.snapshot.parse_predicate_expression(s)?), + Expression::String(s) => { + Some(this.snapshot.parse_predicate_expression(s, &state)?) + } }, None => None, }; @@ -323,17 +312,21 @@ impl std::future::IntoFuture for DeleteBuilder { #[cfg(test)] mod tests { - - use crate::action::*; use crate::operations::DeltaOps; + use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; - use crate::writer::test_utils::{get_arrow_schema, get_delta_schema}; + use crate::writer::test_utils::{ + get_arrow_schema, get_delta_schema, get_record_batch, setup_table_with_configuration, + write_batch, + }; + use crate::DeltaConfigKey; use crate::DeltaTable; use arrow::array::Int32Array; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; use datafusion::assert_batches_sorted_eq; use datafusion::prelude::*; + use serde_json::json; use std::sync::Arc; async fn setup_table(partitions: Option>) -> DeltaTable { @@ -349,6 +342,19 @@ mod tests { table } + #[tokio::test] + async fn test_delete_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 some data + let table = write_batch(table, batch).await; + // delete + let _err = DeltaOps(table) + .delete() + .await + .expect_err("Remove action is included when Delta table is append-only. Should error"); + } + #[tokio::test] async fn test_delete_default() { let schema = get_arrow_schema(&None); @@ -455,7 +461,7 @@ mod tests { assert_eq!(table.version(), 2); assert_eq!(table.get_file_uris().count(), 2); - let (table, metrics) = DeltaOps(table) + let (mut table, metrics) = DeltaOps(table) .delete() .with_predicate(col("value").eq(lit(1))) .await @@ -469,6 +475,11 @@ mod tests { assert_eq!(metrics.num_deleted_rows, Some(1)); 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 parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["predicate"], json!("value = 1")); + let expected = vec![ "+----+-------+------------+", "| id | value | modified |", diff --git a/rust/src/operations/filesystem_check.rs b/crates/deltalake-core/src/operations/filesystem_check.rs similarity index 97% rename from rust/src/operations/filesystem_check.rs rename to crates/deltalake-core/src/operations/filesystem_check.rs index 6cdf5390bf..83af12b57c 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/crates/deltalake-core/src/operations/filesystem_check.rs @@ -22,13 +22,14 @@ use futures::future::BoxFuture; use futures::StreamExt; pub use object_store::path::Path; use object_store::ObjectStore; +use serde::Serialize; use url::{ParseError, Url}; -use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; +use crate::protocol::{Action, Add, DeltaOperation, Remove}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; /// Audit the Delta Table's active files with the underlying file system. @@ -44,7 +45,7 @@ pub struct FileSystemCheckBuilder { } /// Details of the FSCK operation including which files were removed from the log -#[derive(Debug)] +#[derive(Debug, Serialize)] pub struct FileSystemCheckMetrics { /// Was this a dry run pub dry_run: bool, diff --git a/rust/src/operations/load.rs b/crates/deltalake-core/src/operations/load.rs similarity index 99% rename from rust/src/operations/load.rs rename to crates/deltalake-core/src/operations/load.rs index 9501c18011..7baa59e3e1 100644 --- a/rust/src/operations/load.rs +++ b/crates/deltalake-core/src/operations/load.rs @@ -8,7 +8,7 @@ use futures::future::BoxFuture; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; #[derive(Debug, Clone)] diff --git a/rust/src/operations/merge.rs b/crates/deltalake-core/src/operations/merge.rs similarity index 73% rename from rust/src/operations/merge.rs rename to crates/deltalake-core/src/operations/merge.rs index 29c5ffaa27..a51e7649fc 100644 --- a/rust/src/operations/merge.rs +++ b/crates/deltalake-core/src/operations/merge.rs @@ -16,8 +16,9 @@ //! ```rust ignore //! let table = open_table("../path/to/table")?; //! let (table, metrics) = DeltaOps(table) -//! .merge(source, col("id").eq(col("source.id"))) +//! .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") + lit(1)) @@ -32,15 +33,20 @@ //! .await? //! ```` +use std::collections::HashMap; +use std::sync::Arc; +use std::time::{Instant, SystemTime, UNIX_EPOCH}; + use arrow_schema::SchemaRef; use datafusion::error::Result as DataFusionResult; +use datafusion::logical_expr::build_join_schema; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::{ execution::context::SessionState, physical_plan::{ filter::FilterExec, joins::{ - utils::{build_join_schema, JoinFilter}, + utils::{build_join_schema as physical_build_join_schema, JoinFilter}, NestedLoopJoinExec, }, metrics::{MetricBuilder, MetricsSet}, @@ -49,33 +55,27 @@ use datafusion::{ }, prelude::{DataFrame, SessionContext}, }; -use datafusion_common::{Column, DFSchema, ScalarValue}; +use datafusion_common::{Column, DFField, DFSchema, ScalarValue, TableReference}; use datafusion_expr::{col, conditional_expressions::CaseBuilder, lit, when, Expr, JoinType}; use datafusion_physical_expr::{create_physical_expr, expressions, PhysicalExpr}; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; +use serde::Serialize; use serde_json::{Map, Value}; -use std::{ - collections::HashMap, - sync::Arc, - time::{Instant, SystemTime, UNIX_EPOCH}, -}; -use crate::action::MergePredicate; +use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; +use super::transaction::commit; +use crate::delta_datafusion::expr::{fmt_expr_to_sql, parse_predicate_expression}; +use crate::delta_datafusion::{register_store, DeltaScanBuilder}; use crate::operations::datafusion_utils::MetricObserverExec; use crate::{ - action::{Action, DeltaOperation, Remove}, - delta_datafusion::{parquet_scan_from_actions, register_store}, operations::write::write_execution_plan, storage::{DeltaObjectStore, ObjectStoreRef}, - table_state::DeltaTableState, DeltaResult, DeltaTable, DeltaTableError, }; -use super::{ - datafusion_utils::{into_expr, maybe_into_expr, Expression}, - transaction::commit, -}; +use crate::protocol::{Action, DeltaOperation, MergePredicate, Remove}; +use crate::table::state::DeltaTableState; const OPERATION_COLUMN: &str = "__delta_rs_operation"; const DELETE_COLUMN: &str = "__delta_rs_delete"; @@ -95,13 +95,15 @@ pub struct MergeBuilder { /// The join predicate predicate: Expression, /// Operations to perform when a source record and target record match - match_operations: Vec, + match_operations: Vec, /// Operations to perform on source records when they do not pair with a target record - not_match_operations: Vec, + not_match_operations: Vec, /// Operations to perform on target records when they do not pair with a source record - not_match_source_operations: Vec, + not_match_source_operations: Vec, ///Prefix the source columns with a user provided prefix source_alias: Option, + ///Prefix target columns with a user provided prefix + target_alias: Option, /// A snapshot of the table's state. AKA the target table in the operation snapshot: DeltaTableState, /// The source data @@ -121,18 +123,20 @@ pub struct MergeBuilder { impl MergeBuilder { /// Create a new [`MergeBuilder`] - pub fn new( + pub fn new>( object_store: ObjectStoreRef, snapshot: DeltaTableState, - predicate: Expression, + predicate: E, source: DataFrame, ) -> Self { + let predicate = predicate.into(); Self { predicate, source, snapshot, object_store, source_alias: None, + target_alias: None, state: None, app_metadata: None, writer_properties: None, @@ -156,8 +160,9 @@ impl MergeBuilder { /// ```rust ignore /// let table = open_table("../path/to/table")?; /// let (table, metrics) = DeltaOps(table) - /// .merge(source, col("id").eq(col("source.id"))) + /// .merge(source, col("target.id").eq(col("source.id"))) /// .with_source_alias("source") + /// .with_target_alias("target") /// .when_matched_update(|update| { /// update /// .predicate(col("source.value").lt(lit(0))) @@ -176,12 +181,8 @@ impl MergeBuilder { F: FnOnce(UpdateBuilder) -> UpdateBuilder, { let builder = builder(UpdateBuilder::default()); - let op = MergeOperation::try_new( - &self.snapshot, - builder.predicate, - builder.updates, - OperationType::Update, - )?; + let op = + MergeOperationConfig::new(builder.predicate, builder.updates, OperationType::Update)?; self.match_operations.push(op); Ok(self) } @@ -197,8 +198,9 @@ impl MergeBuilder { /// ```rust ignore /// let table = open_table("../path/to/table")?; /// let (table, metrics) = DeltaOps(table) - /// .merge(source, col("id").eq(col("source.id"))) + /// .merge(source, col("target.id").eq(col("source.id"))) /// .with_source_alias("source") + /// .with_target_alias("target") /// .when_matched_delete(|delete| { /// delete.predicate(col("source.delete")) /// })? @@ -209,8 +211,7 @@ impl MergeBuilder { F: FnOnce(DeleteBuilder) -> DeleteBuilder, { let builder = builder(DeleteBuilder::default()); - let op = MergeOperation::try_new( - &self.snapshot, + let op = MergeOperationConfig::new( builder.predicate, HashMap::default(), OperationType::Delete, @@ -230,8 +231,9 @@ impl MergeBuilder { /// ```rust ignore /// let table = open_table("../path/to/table")?; /// let (table, metrics) = DeltaOps(table) - /// .merge(source, col("id").eq(col("source.id"))) + /// .merge(source, col("target.id").eq(col("source.id"))) /// .with_source_alias("source") + /// .with_target_alias("target") /// .when_not_matched_insert(|insert| { /// insert /// .set("id", col("source.id")) @@ -245,12 +247,7 @@ impl MergeBuilder { F: FnOnce(InsertBuilder) -> InsertBuilder, { let builder = builder(InsertBuilder::default()); - let op = MergeOperation::try_new( - &self.snapshot, - builder.predicate, - builder.set, - OperationType::Insert, - )?; + let op = MergeOperationConfig::new(builder.predicate, builder.set, OperationType::Insert)?; self.not_match_operations.push(op); Ok(self) } @@ -269,8 +266,9 @@ impl MergeBuilder { /// ```rust ignore /// let table = open_table("../path/to/table")?; /// let (table, metrics) = DeltaOps(table) - /// .merge(source, col("id").eq(col("source.id"))) + /// .merge(source, col("target.id").eq(col("source.id"))) /// .with_source_alias("source") + /// .with_target_alias("target") /// .when_not_matched_by_source_update(|update| { /// update /// .update("active", lit(false)) @@ -283,12 +281,8 @@ impl MergeBuilder { F: FnOnce(UpdateBuilder) -> UpdateBuilder, { let builder = builder(UpdateBuilder::default()); - let op = MergeOperation::try_new( - &self.snapshot, - builder.predicate, - builder.updates, - OperationType::Update, - )?; + let op = + MergeOperationConfig::new(builder.predicate, builder.updates, OperationType::Update)?; self.not_match_source_operations.push(op); Ok(self) } @@ -304,8 +298,9 @@ impl MergeBuilder { /// ```rust ignore /// let table = open_table("../path/to/table")?; /// let (table, metrics) = DeltaOps(table) - /// .merge(source, col("id").eq(col("source.id"))) + /// .merge(source, col("target.id").eq(col("source.id"))) /// .with_source_alias("source") + /// .with_target_alias("target") /// .when_not_matched_by_source_delete(|delete| { /// delete /// })? @@ -316,8 +311,7 @@ impl MergeBuilder { F: FnOnce(DeleteBuilder) -> DeleteBuilder, { let builder = builder(DeleteBuilder::default()); - let op = MergeOperation::try_new( - &self.snapshot, + let op = MergeOperationConfig::new( builder.predicate, HashMap::default(), OperationType::Delete, @@ -332,6 +326,12 @@ impl MergeBuilder { self } + /// Rename columns in the target dataset to have a prefix of `alias`.`original column name` + pub fn with_target_alias(mut self, alias: S) -> Self { + self.target_alias = Some(alias.to_string()); + self + } + /// The Datafusion session state to use pub fn with_session_state(mut self, state: SessionState) -> Self { self.state = Some(state); @@ -444,6 +444,15 @@ enum OperationType { Copy, } +//Encapsute the User's Merge configuration for later processing +struct MergeOperationConfig { + /// Which records to update + predicate: Option, + /// How to update columns in a record that match the predicate + operations: HashMap, + r#type: OperationType, +} + struct MergeOperation { /// Which records to update predicate: Option, @@ -453,28 +462,77 @@ struct MergeOperation { } impl MergeOperation { - pub fn try_new( - snapshot: &DeltaTableState, + fn try_from( + config: MergeOperationConfig, + schema: &DFSchema, + state: &SessionState, + target_alias: &Option, + ) -> DeltaResult { + let mut ops = HashMap::with_capacity(config.operations.capacity()); + + for (column, expression) in config.operations.into_iter() { + // Normalize the column name to contain the target alias. If a table reference was provided ensure it's the target. + let column = match target_alias { + Some(alias) => { + let r = TableReference::bare(alias.to_owned()); + match column { + Column { + relation: None, + name, + } => Column { + relation: Some(r), + name, + }, + Column { + relation: Some(TableReference::Bare { table }), + name, + } => { + if table.eq(alias) { + Column { + relation: Some(r), + name, + } + } else { + return Err(DeltaTableError::Generic( + format!("Table alias '{table}' in column reference '{table}.{name}' unknown. Hint: You must reference the Delta Table with alias '{alias}'.") + )); + } + } + _ => { + return Err(DeltaTableError::Generic( + "Column must reference column in Delta table".into(), + )) + } + } + } + None => column, + }; + ops.insert(column, into_expr(expression, schema, state)?); + } + + Ok(MergeOperation { + predicate: maybe_into_expr(config.predicate, schema, state)?, + operations: ops, + r#type: config.r#type, + }) + } +} + +impl MergeOperationConfig { + pub fn new( predicate: Option, operations: HashMap, r#type: OperationType, ) -> DeltaResult { - let predicate = maybe_into_expr(predicate, snapshot)?; - let mut _operations = HashMap::new(); - - for (column, expr) in operations { - _operations.insert(column, into_expr(expr, snapshot)?); - } - - Ok(MergeOperation { + Ok(MergeOperationConfig { predicate, - operations: _operations, + operations, r#type, }) } } -#[derive(Default)] +#[derive(Default, Serialize, Debug)] /// Metrics for the Merge Operation pub struct MergeMetrics { /// Number of rows in the source data @@ -512,9 +570,10 @@ async fn execute( app_metadata: Option>, safe_cast: bool, source_alias: Option, - match_operations: Vec, - not_match_target_operations: Vec, - not_match_source_operations: Vec, + target_alias: Option, + match_operations: Vec, + not_match_target_operations: Vec, + not_match_source_operations: Vec, ) -> DeltaResult<((Vec, i64), MergeMetrics)> { let mut metrics = MergeMetrics::default(); let exec_start = Instant::now(); @@ -523,13 +582,6 @@ async fn execute( .current_metadata() .ok_or(DeltaTableError::NoMetadata)?; - let predicate = match predicate { - Expression::DataFusion(expr) => expr, - Expression::String(s) => snapshot.parse_predicate_expression(s)?, - }; - - let schema = snapshot.input_schema()?; - // TODO: Given the join predicate, remove any expression that involve the // source table and keep expressions that only involve the target table. // This would allow us to perform statistics/partition pruning E.G @@ -539,17 +591,12 @@ async fn execute( // If the user specified any not_source_match operations then those // predicates also need to be considered when pruning - let target = parquet_scan_from_actions( - snapshot, - object_store.clone(), - snapshot.files(), - &schema, - None, - &state, - None, - None, - ) - .await?; + let target = Arc::new( + DeltaScanBuilder::new(snapshot, object_store.clone(), &state) + .with_schema(snapshot.input_schema()?) + .build() + .await?, + ); let source = source.create_physical_plan().await?; @@ -562,16 +609,10 @@ async fn execute( let mut expressions: Vec<(Arc, String)> = Vec::new(); let source_schema = source_count.schema(); - let source_prefix = source_alias - .map(|mut s| { - s.push('.'); - s - }) - .unwrap_or_default(); for (i, field) in source_schema.fields().into_iter().enumerate() { expressions.push(( Arc::new(expressions::Column::new(field.name(), i)), - source_prefix.clone() + field.name(), + field.name().clone(), )); } expressions.push(( @@ -602,15 +643,54 @@ async fn execute( let target = Arc::new(CoalescePartitionsExec::new(target)); let source = Arc::new(CoalescePartitionsExec::new(source)); - let join_schema = build_join_schema(&source.schema(), &target.schema(), &JoinType::Full); + let source_schema = match &source_alias { + Some(alias) => { + DFSchema::try_from_qualified_schema(TableReference::bare(alias), &source.schema())? + } + None => DFSchema::try_from(source.schema().as_ref().to_owned())?, + }; + + let target_schema = match &target_alias { + Some(alias) => { + DFSchema::try_from_qualified_schema(TableReference::bare(alias), &target.schema())? + } + None => DFSchema::try_from(target.schema().as_ref().to_owned())?, + }; + + let join_schema_df = build_join_schema(&source_schema, &target_schema, &JoinType::Full)?; + + let join_schema = + physical_build_join_schema(&source.schema(), &target.schema(), &JoinType::Full); + let (join_schema, join_order) = (join_schema.0, join_schema.1); + + let predicate = match predicate { + Expression::DataFusion(expr) => expr, + Expression::String(s) => parse_predicate_expression(&join_schema_df, s, &state)?, + }; + + let match_operations: Vec = match_operations + .into_iter() + .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) + .collect::, DeltaTableError>>()?; + + let not_match_target_operations: Vec = not_match_target_operations + .into_iter() + .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) + .collect::, DeltaTableError>>()?; + + let not_match_source_operations: Vec = not_match_source_operations + .into_iter() + .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) + .collect::, DeltaTableError>>()?; + let predicate_expr = create_physical_expr( &predicate, - &join_schema.0.clone().try_into()?, - &join_schema.0, + &join_schema_df, + &join_schema, state.execution_props(), )?; - let join_filter = JoinFilter::new(predicate_expr, join_schema.1, join_schema.0); + let join_filter = JoinFilter::new(predicate_expr, join_order, join_schema); let join: Arc = Arc::new(NestedLoopJoinExec::try_new( source.clone(), target.clone(), @@ -682,7 +762,10 @@ async fn execute( }; let action_type = action_type.to_string(); - let predicate = op.predicate.map(|expr| expr.display_name().unwrap()); + let predicate = op + .predicate + .map(|expr| fmt_expr_to_sql(&expr)) + .transpose()?; predicates.push(MergePredicate { action_type, @@ -732,13 +815,21 @@ async fn execute( let case = create_physical_expr( &case, - &join.schema().as_ref().to_owned().try_into()?, + &join_schema_df, &join.schema(), state.execution_props(), )?; expressions.push((case, OPERATION_COLUMN.to_owned())); let projection = Arc::new(ProjectionExec::try_new(expressions, join.clone())?); + let mut f = join_schema_df.fields().to_owned(); + f.push(DFField::new_unqualified( + OPERATION_COLUMN, + arrow_schema::DataType::Int64, + false, + )); + let project_schema_df = DFSchema::new_with_metadata(f, HashMap::new())?; + // Project again and include the original table schema plus a column to mark if row needs to be filtered before write let mut expressions: Vec<(Arc, String)> = Vec::new(); let schema = projection.schema(); @@ -750,15 +841,27 @@ async fn execute( } let mut projection_map = HashMap::new(); - for field in snapshot.schema().unwrap().get_fields() { + let mut f = project_schema_df.fields().clone(); + + for delta_field in snapshot.schema().unwrap().get_fields() { let mut when_expr = Vec::with_capacity(operations_size); let mut then_expr = Vec::with_capacity(operations_size); + let qualifier = match &target_alias { + Some(alias) => Some(TableReference::Bare { + table: alias.to_owned().into(), + }), + None => TableReference::none(), + }; + let name = delta_field.get_name(); + let column = Column::new(qualifier.clone(), name); + let field = project_schema_df.field_with_name(qualifier.as_ref(), name)?; + for (idx, (operations, _)) in ops.iter().enumerate() { let op = operations - .get(&field.get_name().to_owned().into()) + .get(&column) .map(|expr| expr.to_owned()) - .unwrap_or(col(field.get_name())); + .unwrap_or_else(|| col(column.clone())); when_expr.push(lit(idx as i32)); then_expr.push(op); @@ -774,13 +877,20 @@ async fn execute( let case = create_physical_expr( &case, - &projection.schema().as_ref().to_owned().try_into()?, + &project_schema_df, &projection.schema(), state.execution_props(), )?; - projection_map.insert(field.get_name(), expressions.len()); - expressions.push((case, "__delta_rs_c_".to_owned() + field.get_name())); + projection_map.insert(delta_field.get_name(), expressions.len()); + let name = "__delta_rs_c_".to_owned() + delta_field.get_name(); + + f.push(DFField::new_unqualified( + &name, + field.data_type().clone(), + true, + )); + expressions.push((case, name)); } let mut insert_when = Vec::with_capacity(ops.len()); @@ -865,7 +975,7 @@ async fn execute( } let schema = projection.schema(); - let input_dfschema = schema.as_ref().to_owned().try_into()?; + let input_dfschema = project_schema_df; expressions.push(( build_case( delete_when, @@ -876,6 +986,11 @@ async fn execute( )?, DELETE_COLUMN.to_owned(), )); + f.push(DFField::new_unqualified( + DELETE_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); expressions.push(( build_case( @@ -887,6 +1002,12 @@ async fn execute( )?, TARGET_INSERT_COLUMN.to_owned(), )); + f.push(DFField::new_unqualified( + TARGET_INSERT_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + expressions.push(( build_case( update_when, @@ -897,6 +1018,12 @@ async fn execute( )?, TARGET_UPDATE_COLUMN.to_owned(), )); + f.push(DFField::new_unqualified( + TARGET_UPDATE_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + expressions.push(( build_case( target_delete_when, @@ -907,6 +1034,12 @@ async fn execute( )?, TARGET_DELETE_COLUMN.to_owned(), )); + f.push(DFField::new_unqualified( + TARGET_DELETE_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + expressions.push(( build_case( copy_when, @@ -917,6 +1050,11 @@ async fn execute( )?, TARGET_COPY_COLUMN.to_owned(), )); + f.push(DFField::new_unqualified( + TARGET_COPY_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); let projection = Arc::new(ProjectionExec::try_new(expressions, projection.clone())?); @@ -955,9 +1093,11 @@ async fn execute( ); })); + let write_schema_df = DFSchema::new_with_metadata(f, HashMap::new())?; + let write_predicate = create_physical_expr( &(col(DELETE_COLUMN).is_false()), - &target_count_plan.schema().as_ref().to_owned().try_into()?, + &write_schema_df, &target_count_plan.schema(), state.execution_props(), )?; @@ -1042,7 +1182,7 @@ async fn execute( // Do not make a commit when there are zero updates to the state if !actions.is_empty() { let operation = DeltaOperation::Merge { - predicate: Some(predicate.canonical_name()), + 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, @@ -1087,6 +1227,7 @@ impl std::future::IntoFuture for MergeBuilder { this.app_metadata, this.safe_cast, this.source_alias, + this.target_alias, this.match_operations, this.not_match_operations, this.not_match_source_operations, @@ -1104,22 +1245,26 @@ impl std::future::IntoFuture for MergeBuilder { #[cfg(test)] mod tests { - - use crate::action::*; use crate::operations::DeltaOps; + use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::get_arrow_schema; use crate::writer::test_utils::get_delta_schema; + use crate::writer::test_utils::setup_table_with_configuration; + use crate::DeltaConfigKey; use crate::DeltaTable; use arrow::datatypes::Schema as ArrowSchema; use arrow::record_batch::RecordBatch; use datafusion::assert_batches_sorted_eq; + use datafusion::prelude::DataFrame; use datafusion::prelude::SessionContext; use datafusion_expr::col; use datafusion_expr::lit; use serde_json::json; use std::sync::Arc; + use super::MergeMetrics; + async fn setup_table(partitions: Option>) -> DeltaTable { let table_schema = get_delta_schema(); @@ -1133,6 +1278,21 @@ mod tests { table } + #[tokio::test] + async fn test_merge_when_delta_table_is_append_only() { + let schema = get_arrow_schema(&None); + let table = setup_table_with_configuration(DeltaConfigKey::AppendOnly, Some("true")).await; + // append some data + let table = write_data(table, &schema).await; + // merge + let _err = DeltaOps(table) + .merge(merge_source(schema), col("target.id").eq(col("source.id"))) + .with_source_alias("source") + .with_target_alias("target") + .await + .expect_err("Remove action is included when Delta table is append-only. Should error"); + } + async fn write_data(table: DeltaTable, schema: &Arc) -> DeltaTable { let batch = RecordBatch::try_new( Arc::clone(schema), @@ -1156,15 +1316,7 @@ mod tests { .unwrap() } - #[tokio::test] - async fn test_merge() { - let schema = get_arrow_schema(&None); - let table = setup_table(None).await; - - let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); - assert_eq!(table.get_file_uris().count(), 1); - + fn merge_source(schema: Arc) -> DataFrame { let ctx = SessionContext::new(); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -1179,11 +1331,55 @@ mod tests { ], ) .unwrap(); - let source = ctx.read_batch(batch).unwrap(); + ctx.read_batch(batch).unwrap() + } + + async fn setup() -> (DeltaTable, DataFrame) { + let schema = get_arrow_schema(&None); + let table = setup_table(None).await; + + let table = write_data(table, &schema).await; + assert_eq!(table.version(), 1); + assert_eq!(table.get_file_uris().count(), 1); + + (table, merge_source(schema)) + } + + async fn assert_merge(table: DeltaTable, metrics: MergeMetrics) { + assert_eq!(table.version(), 2); + assert_eq!(table.get_file_uris().count(), 1); + 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, 3); + assert_eq!(metrics.num_target_rows_inserted, 1); + assert_eq!(metrics.num_target_rows_deleted, 0); + assert_eq!(metrics.num_output_rows, 5); + assert_eq!(metrics.num_source_rows, 3); + + let expected = vec![ + "+----+-------+------------+", + "| id | value | modified |", + "+----+-------+------------+", + "| A | 2 | 2021-02-01 |", + "| B | 10 | 2021-02-02 |", + "| C | 20 | 2023-07-04 |", + "| D | 100 | 2021-02-02 |", + "| X | 30 | 2023-07-04 |", + "+----+-------+------------+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } + + #[tokio::test] + async fn test_merge() { + let (table, source) = setup().await; let (mut table, metrics) = DeltaOps(table) - .merge(source, col("id").eq(col("source.id"))) + .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")) @@ -1192,8 +1388,8 @@ mod tests { .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate(col("value").eq(lit(1))) - .update("value", col("value") + lit(1)) + .predicate(col("target.value").eq(lit(1))) + .update("value", col("target.value") + lit(1)) }) .unwrap() .when_not_matched_insert(|insert| { @@ -1206,21 +1402,62 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); - 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, 3); - assert_eq!(metrics.num_target_rows_inserted, 1); - assert_eq!(metrics.num_target_rows_deleted, 0); - assert_eq!(metrics.num_output_rows, 5); - 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 parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["predicate"], json!("target.id = source.id")); + assert_eq!( + parameters["matchedPredicates"], + json!(r#"[{"actionType":"update"}]"#) + ); + assert_eq!( + parameters["notMatchedPredicates"], + json!(r#"[{"actionType":"insert"}]"#) + ); + assert_eq!( + parameters["notMatchedBySourcePredicates"], + json!(r#"[{"actionType":"update","predicate":"target.value = 1"}]"#) + ); + + assert_merge(table, metrics).await; + } + + #[tokio::test] + async fn test_merge_str() { + // Validate that users can use string predicates + // Also validates that update and set operations can contain the target alias + let (table, source) = setup().await; + + let (mut table, metrics) = DeltaOps(table) + .merge(source, "target.id = source.id") + .with_source_alias("source") + .with_target_alias("target") + .when_matched_update(|update| { + update + .update("target.value", "source.value") + .update("modified", "source.modified") + }) + .unwrap() + .when_not_matched_by_source_update(|update| { + update + .predicate("target.value = arrow_cast(1, 'Int32')") + .update("value", "target.value + cast(1 as int)") + }) + .unwrap() + .when_not_matched_insert(|insert| { + insert + .set("target.id", "source.id") + .set("value", "source.value") + .set("modified", "source.modified") + }) + .unwrap() + .await + .unwrap(); let commit_info = table.history(None).await.unwrap(); let last_commit = &commit_info[commit_info.len() - 1]; let parameters = last_commit.operation_parameters.clone().unwrap(); - assert_eq!(parameters["predicate"], json!("id = source.id")); + assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( parameters["matchedPredicates"], json!(r#"[{"actionType":"update"}]"#) @@ -1229,25 +1466,129 @@ mod tests { parameters["notMatchedPredicates"], json!(r#"[{"actionType":"insert"}]"#) ); - // Todo: Expected this predicate to actually be 'value = 1'. Predicate should contain a valid sql expression assert_eq!( parameters["notMatchedBySourcePredicates"], - json!(r#"[{"actionType":"update","predicate":"value = Int32(1)"}]"#) + json!( + r#"[{"actionType":"update","predicate":"target.value = arrow_cast(1, 'Int32')"}]"# + ) ); - let expected = vec![ - "+----+-------+------------+", - "| id | value | modified |", - "+----+-------+------------+", - "| A | 2 | 2021-02-01 |", - "| B | 10 | 2021-02-02 |", - "| C | 20 | 2023-07-04 |", - "| D | 100 | 2021-02-02 |", - "| X | 30 | 2023-07-04 |", - "+----+-------+------------+", - ]; - let actual = get_data(&table).await; - assert_batches_sorted_eq!(&expected, &actual); + assert_merge(table, metrics).await; + } + + #[tokio::test] + async fn test_merge_no_alias() { + // Validate merge can be used without specifying an alias + let (table, source) = setup().await; + + let source = source + .with_column_renamed("id", "source_id") + .unwrap() + .with_column_renamed("value", "source_value") + .unwrap() + .with_column_renamed("modified", "source_modified") + .unwrap(); + + let (table, metrics) = DeltaOps(table) + .merge(source, "id = source_id") + .when_matched_update(|update| { + update + .update("value", "source_value") + .update("modified", "source_modified") + }) + .unwrap() + .when_not_matched_by_source_update(|update| { + update + .predicate("value = arrow_cast(1, 'Int32')") + .update("value", "value + cast(1 as int)") + }) + .unwrap() + .when_not_matched_insert(|insert| { + insert + .set("id", "source_id") + .set("value", "source_value") + .set("modified", "source_modified") + }) + .unwrap() + .await + .unwrap(); + + assert_merge(table, metrics).await; + } + + #[tokio::test] + async fn test_merge_with_alias_mix() { + // Validate merge can be used with an alias and unambiguous column references + // I.E users should be able to specify an alias and still reference columns without using that alias when there is no ambiguity + let (table, source) = setup().await; + + let source = source + .with_column_renamed("id", "source_id") + .unwrap() + .with_column_renamed("value", "source_value") + .unwrap() + .with_column_renamed("modified", "source_modified") + .unwrap(); + + let (table, metrics) = DeltaOps(table) + .merge(source, "id = source_id") + .with_target_alias("target") + .when_matched_update(|update| { + update + .update("value", "source_value") + .update("modified", "source_modified") + }) + .unwrap() + .when_not_matched_by_source_update(|update| { + update + .predicate("value = arrow_cast(1, 'Int32')") + .update("value", "target.value + cast(1 as int)") + }) + .unwrap() + .when_not_matched_insert(|insert| { + insert + .set("id", "source_id") + .set("target.value", "source_value") + .set("modified", "source_modified") + }) + .unwrap() + .await + .unwrap(); + + assert_merge(table, metrics).await; + } + + #[tokio::test] + async fn test_merge_failures() { + // Validate target columns cannot be from the source + let (table, source) = setup().await; + let res = 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("source.value", "source.value") + .update("modified", "source.modified") + }) + .unwrap() + .await; + assert!(res.is_err()); + + // Validate failure when aliases are the same + let (table, source) = setup().await; + let res = DeltaOps(table) + .merge(source, col("target.id").eq(col("source.id"))) + .with_source_alias("source") + .with_target_alias("source") + .when_matched_update(|update| { + update + .update("target.value", "source.value") + .update("modified", "source.modified") + }) + .unwrap() + .await; + assert!(res.is_err()) } #[tokio::test] @@ -1279,11 +1620,12 @@ mod tests { let (table, metrics) = DeltaOps(table) .merge( source, - col("id") + col("target.id") .eq(col("source.id")) - .and(col("modified").eq(lit("2021-02-02"))), + .and(col("target.modified").eq(lit("2021-02-02"))), ) .with_source_alias("source") + .with_target_alias("target") .when_matched_update(|update| { update .update("value", col("source.value")) @@ -1292,14 +1634,14 @@ mod tests { .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate(col("value").eq(lit(1))) - .update("value", col("value") + lit(1)) + .predicate(col("target.value").eq(lit(1))) + .update("value", col("target.value") + lit(1)) }) .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate(col("modified").eq(lit("2021-02-01"))) - .update("value", col("value") - lit(1)) + .predicate(col("target.modified").eq(lit("2021-02-01"))) + .update("value", col("target.value") - lit(1)) }) .unwrap() .when_not_matched_insert(|insert| { @@ -1367,8 +1709,9 @@ mod tests { let source = ctx.read_batch(batch).unwrap(); let (mut table, metrics) = DeltaOps(table) - .merge(source, col("id").eq(col("source.id"))) + .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") + .with_target_alias("target") .when_matched_delete(|delete| delete) .unwrap() .await @@ -1388,7 +1731,7 @@ mod tests { let commit_info = table.history(None).await.unwrap(); let last_commit = &commit_info[commit_info.len() - 1]; let parameters = last_commit.operation_parameters.clone().unwrap(); - assert_eq!(parameters["predicate"], json!("id = source.id")); + assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( parameters["matchedPredicates"], json!(r#"[{"actionType":"delete"}]"#) @@ -1430,8 +1773,9 @@ mod tests { let source = ctx.read_batch(batch).unwrap(); let (mut table, metrics) = DeltaOps(table) - .merge(source, col("id").eq(col("source.id"))) + .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") + .with_target_alias("target") .when_matched_delete(|delete| delete.predicate(col("source.value").lt_eq(lit(10)))) .unwrap() .await @@ -1451,10 +1795,10 @@ mod tests { let commit_info = table.history(None).await.unwrap(); let last_commit = &commit_info[commit_info.len() - 1]; let parameters = last_commit.operation_parameters.clone().unwrap(); - assert_eq!(parameters["predicate"], json!("id = source.id")); + assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( parameters["matchedPredicates"], - json!(r#"[{"actionType":"delete","predicate":"source.value <= Int32(10)"}]"#) + json!(r#"[{"actionType":"delete","predicate":"source.value <= 10"}]"#) ); let expected = vec![ @@ -1498,8 +1842,9 @@ mod tests { let source = ctx.read_batch(batch).unwrap(); let (mut table, metrics) = DeltaOps(table) - .merge(source, col("id").eq(col("source.id"))) + .merge(source, 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 @@ -1519,7 +1864,7 @@ mod tests { let commit_info = table.history(None).await.unwrap(); let last_commit = &commit_info[commit_info.len() - 1]; let parameters = last_commit.operation_parameters.clone().unwrap(); - assert_eq!(parameters["predicate"], json!("id = source.id")); + assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( parameters["notMatchedBySourcePredicates"], json!(r#"[{"actionType":"delete"}]"#) @@ -1560,10 +1905,11 @@ mod tests { let source = ctx.read_batch(batch).unwrap(); let (mut table, metrics) = DeltaOps(table) - .merge(source, col("id").eq(col("source.id"))) + .merge(source, col("target.id").eq(col("source.id"))) .with_source_alias("source") + .with_target_alias("target") .when_not_matched_by_source_delete(|delete| { - delete.predicate(col("modified").gt(lit("2021-02-01"))) + delete.predicate(col("target.modified").gt(lit("2021-02-01"))) }) .unwrap() .await @@ -1583,10 +1929,10 @@ mod tests { let commit_info = table.history(None).await.unwrap(); let last_commit = &commit_info[commit_info.len() - 1]; let parameters = last_commit.operation_parameters.clone().unwrap(); - assert_eq!(parameters["predicate"], json!("id = source.id")); + assert_eq!(parameters["predicate"], json!("target.id = source.id")); assert_eq!( parameters["notMatchedBySourcePredicates"], - json!(r#"[{"actionType":"delete","predicate":"modified > Utf8(\"2021-02-01\")"}]"#) + json!(r#"[{"actionType":"delete","predicate":"target.modified > '2021-02-01'"}]"#) ); let expected = vec![ diff --git a/rust/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs similarity index 94% rename from rust/src/operations/mod.rs rename to crates/deltalake-core/src/operations/mod.rs index 28dea06777..35301f067e 100644 --- a/rust/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -10,8 +10,8 @@ use self::create::CreateBuilder; use self::filesystem_check::FileSystemCheckBuilder; use self::vacuum::VacuumBuilder; -use crate::builder::DeltaTableBuilder; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::table::builder::DeltaTableBuilder; use crate::DeltaTable; pub mod create; @@ -60,7 +60,7 @@ impl DeltaOps { /// Create a new [`DeltaOps`] instance, operating on [`DeltaTable`] at given uri. /// /// ``` - /// use deltalake::DeltaOps; + /// use deltalake_core::DeltaOps; /// /// async { /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); @@ -82,7 +82,7 @@ impl DeltaOps { /// The main purpose of in-memory tables is for use in testing. /// /// ``` - /// use deltalake::DeltaOps; + /// use deltalake_core::DeltaOps; /// /// let ops = DeltaOps::new_in_memory(); /// ``` @@ -97,7 +97,7 @@ impl DeltaOps { /// Create a new Delta table /// /// ``` - /// use deltalake::DeltaOps; + /// use deltalake_core::DeltaOps; /// /// async { /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); @@ -205,15 +205,17 @@ mod datafusion_utils { use arrow_schema::SchemaRef; use datafusion::arrow::record_batch::RecordBatch; use datafusion::error::Result as DataFusionResult; + use datafusion::execution::context::SessionState; use datafusion::physical_plan::DisplayAs; use datafusion::physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricsSet}, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; + use datafusion_common::DFSchema; use datafusion_expr::Expr; use futures::{Stream, StreamExt}; - use crate::{table_state::DeltaTableState, DeltaResult}; + use crate::{delta_datafusion::expr::parse_predicate_expression, DeltaResult}; /// Used to represent user input of either a Datafusion expression or string expression pub enum Expression { @@ -240,19 +242,24 @@ mod datafusion_utils { } } - pub(crate) fn into_expr(expr: Expression, snapshot: &DeltaTableState) -> DeltaResult { + pub(crate) fn into_expr( + expr: Expression, + schema: &DFSchema, + df_state: &SessionState, + ) -> DeltaResult { match expr { Expression::DataFusion(expr) => Ok(expr), - Expression::String(s) => snapshot.parse_predicate_expression(s), + Expression::String(s) => parse_predicate_expression(schema, s, df_state), } } pub(crate) fn maybe_into_expr( expr: Option, - snapshot: &DeltaTableState, + schema: &DFSchema, + df_state: &SessionState, ) -> DeltaResult> { Ok(match expr { - Some(predicate) => Some(into_expr(predicate, snapshot)?), + Some(predicate) => Some(into_expr(predicate, schema, df_state)?), None => None, }) } diff --git a/rust/src/operations/optimize.rs b/crates/deltalake-core/src/operations/optimize.rs similarity index 90% rename from rust/src/operations/optimize.rs rename to crates/deltalake-core/src/operations/optimize.rs index b0321aa434..ae9ab6cd65 100644 --- a/rust/src/operations/optimize.rs +++ b/crates/deltalake-core/src/operations/optimize.rs @@ -41,10 +41,10 @@ use serde_json::Map; use super::transaction::commit; use super::writer::{PartitionWriter, PartitionWriterConfig}; -use crate::action::{self, Action, DeltaOperation}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{self, Action, DeltaOperation}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; use crate::{crate_version, DeltaTable, ObjectMeta, PartitionFilter}; @@ -157,7 +157,7 @@ pub struct OptimizeBuilder<'a> { /// Delta object store for handling data files store: ObjectStoreRef, /// Filters to select specific table partitions to be optimized - filters: &'a [PartitionFilter<'a, &'a str>], + filters: &'a [PartitionFilter], /// Desired file size after bin-packing files target_size: Option, /// Properties passed to underlying parquet writer @@ -200,7 +200,7 @@ impl<'a> OptimizeBuilder<'a> { } /// Only optimize files that return true for the specified partition filter - pub fn with_filters(mut self, filters: &'a [PartitionFilter<'a, &'a str>]) -> Self { + pub fn with_filters(mut self, filters: &'a [PartitionFilter]) -> Self { self.filters = filters; self } @@ -311,7 +311,7 @@ fn create_remove( let deletion_time = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); let deletion_time = deletion_time.as_millis() as i64; - Ok(Action::remove(action::Remove { + Ok(Action::remove(protocol::Remove { path: path.to_string(), deletion_timestamp: Some(deletion_time), data_change: false, @@ -769,7 +769,7 @@ impl PartitionTuples { pub fn create_merge_plan( optimize_type: OptimizeType, snapshot: &DeltaTableState, - filters: &[PartitionFilter<'_, &str>], + filters: &[PartitionFilter], target_size: Option, writer_properties: WriterProperties, ) -> Result { @@ -860,7 +860,7 @@ impl IntoIterator for MergeBin { fn build_compaction_plan( snapshot: &DeltaTableState, partition_keys: &[String], - filters: &[PartitionFilter<'_, &str>], + filters: &[PartitionFilter], target_size: i64, ) -> Result<(OptimizeOperations, Metrics), DeltaTableError> { let mut metrics = Metrics::default(); @@ -876,10 +876,7 @@ fn build_compaction_plan( let part = PartitionTuples::from_hashmap(partition_keys, &add.partition_values); - partition_files - .entry(part) - .or_insert_with(Vec::new) - .push(object_meta); + partition_files.entry(part).or_default().push(object_meta); } for file in partition_files.values_mut() { @@ -926,7 +923,7 @@ fn build_zorder_plan( zorder_columns: Vec, snapshot: &DeltaTableState, partition_keys: &[String], - filters: &[PartitionFilter<'_, &str>], + filters: &[PartitionFilter], ) -> Result<(OptimizeOperations, Metrics), DeltaTableError> { if zorder_columns.is_empty() { return Err(DeltaTableError::Generic( @@ -1143,7 +1140,7 @@ pub(super) mod zorder { runtime.register_object_store(&Url::parse("delta-rs://").unwrap(), object_store); use url::Url; - let ctx = SessionContext::with_config_rt(SessionConfig::default(), runtime); + let ctx = SessionContext::new_with_config_rt(SessionConfig::default(), runtime); ctx.register_udf(datafusion::zorder_key_udf()); Ok(Self { columns, ctx }) } @@ -1184,6 +1181,106 @@ pub(super) mod zorder { let array = zorder_key(&columns)?; Ok(ColumnarValue::Array(array)) } + + #[cfg(test)] + mod tests { + use super::*; + use ::datafusion::assert_batches_eq; + use arrow_array::{Int32Array, StringArray}; + use arrow_ord::sort::sort_to_indices; + use arrow_select::take::take; + use rand::Rng; + #[test] + fn test_order() { + let int: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); + let str: ArrayRef = Arc::new(StringArray::from(vec![ + Some("a"), + Some("x"), + Some("a"), + Some("x"), + None, + ])); + let int_large: ArrayRef = Arc::new(Int32Array::from(vec![10000, 2000, 300, 40, 5])); + let batch = RecordBatch::try_from_iter(vec![ + ("int", int), + ("str", str), + ("int_large", int_large), + ]) + .unwrap(); + + let expected_1 = vec![ + "+-----+-----+-----------+", + "| int | str | int_large |", + "+-----+-----+-----------+", + "| 1 | a | 10000 |", + "| 2 | x | 2000 |", + "| 3 | a | 300 |", + "| 4 | x | 40 |", + "| 5 | | 5 |", + "+-----+-----+-----------+", + ]; + let expected_2 = vec![ + "+-----+-----+-----------+", + "| int | str | int_large |", + "+-----+-----+-----------+", + "| 5 | | 5 |", + "| 1 | a | 10000 |", + "| 3 | a | 300 |", + "| 2 | x | 2000 |", + "| 4 | x | 40 |", + "+-----+-----+-----------+", + ]; + let expected_3 = vec![ + "+-----+-----+-----------+", + "| int | str | int_large |", + "+-----+-----+-----------+", + "| 5 | | 5 |", + "| 4 | x | 40 |", + "| 2 | x | 2000 |", + "| 3 | a | 300 |", + "| 1 | a | 10000 |", + "+-----+-----+-----------+", + ]; + + let expected = vec![expected_1, expected_2, expected_3]; + + let indices = Int32Array::from(shuffled_indices().to_vec()); + let shuffled_columns = batch + .columns() + .iter() + .map(|c| take(c, &indices, None).unwrap()) + .collect::>(); + let shuffled_batch = + RecordBatch::try_new(batch.schema(), shuffled_columns).unwrap(); + + for i in 1..=batch.num_columns() { + let columns = (0..i) + .map(|idx| shuffled_batch.column(idx).clone()) + .collect::>(); + + let order_keys = zorder_key(&columns).unwrap(); + let indices = sort_to_indices(order_keys.as_ref(), None, None).unwrap(); + let sorted_columns = shuffled_batch + .columns() + .iter() + .map(|c| take(c, &indices, None).unwrap()) + .collect::>(); + let sorted_batch = + RecordBatch::try_new(batch.schema(), sorted_columns).unwrap(); + + assert_batches_eq!(expected[i - 1], &[sorted_batch]); + } + } + fn shuffled_indices() -> [i32; 5] { + let mut rng = rand::thread_rng(); + let mut array = [0, 1, 2, 3, 4]; + for i in (1..array.len()).rev() { + let j = rng.gen_range(0..=i); + array.swap(i, j); + } + array + } + } } /// Creates a new binary array containing the zorder keys for the given columns @@ -1207,7 +1304,7 @@ pub(super) mod zorder { )); } - // We are taking 128 bits from each value. Shorter values will be padded. + // We are taking 128 bits (16 bytes) from each value. Shorter values will be padded. let value_size: usize = columns.len() * 16; // Initialize with zeros @@ -1245,7 +1342,7 @@ pub(super) mod zorder { out: &mut Vec, ) -> Result<(), ArrowError> { // Convert array to rows - let mut converter = RowConverter::new(vec![SortField::new(input.data_type().clone())])?; + let converter = RowConverter::new(vec![SortField::new(input.data_type().clone())])?; let rows = converter.convert_columns(&[input])?; for (row_i, row) in rows.iter().enumerate() { @@ -1337,27 +1434,6 @@ pub(super) mod zorder { let data: &BinaryArray = as_generic_binary_array(result.as_ref()); assert_eq!(data.value_data().len(), 3 * 16 * 3); assert!(data.iter().all(|x| x.unwrap().len() == 3 * 16)); - - // This value is mostly filled in since it has a large string - assert_eq!( - data.value(0), - [ - 28, 0, 0, 133, 128, 13, 130, 0, 9, 128, 4, 9, 128, 32, 9, 2, 0, 9, 130, 4, 1, - 16, 32, 9, 18, 32, 9, 16, 36, 1, 0, 0, 1, 2, 0, 8, 0, 0, 1, 144, 4, 9, 2, 0, 9, - 128, 32, 9, - ] - ); - - // This value only has short strings, so it's largely zeros - assert_eq!( - data.value(1)[0..12], - [28u8, 0u8, 0u8, 26, 129, 13, 2, 0, 9, 128, 32, 9] - ); - assert_eq!(data.value(1)[12..], [0; (3 * 16) - 12]); - - // Last value is all nulls, so mostly zeros - assert_eq!(data.value(2)[0..1], [2u8]); - assert_eq!(data.value(2)[1..], [0; (3 * 16) - 1]); } } } diff --git a/rust/src/operations/restore.rs b/crates/deltalake-core/src/operations/restore.rs similarity index 92% rename from rust/src/operations/restore.rs rename to crates/deltalake-core/src/operations/restore.rs index a5e51da05b..1f4de3a06c 100644 --- a/rust/src/operations/restore.rs +++ b/crates/deltalake-core/src/operations/restore.rs @@ -30,11 +30,11 @@ use object_store::path::Path; use object_store::ObjectStore; use serde::Serialize; -use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::operations::transaction::{prepare_commit, try_commit_transaction, TransactionError}; +use crate::protocol::{Action, Add, DeltaOperation, Protocol, Remove}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; -use crate::{action, DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; +use crate::table::state::DeltaTableState; +use crate::{DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; /// Errors that can occur during restore #[derive(thiserror::Error, Debug)] @@ -202,12 +202,22 @@ async fn execute( let mut actions = vec![]; let protocol = if protocol_downgrade_allowed { - action::Protocol { + Protocol { min_reader_version: table.get_min_reader_version(), min_writer_version: table.get_min_writer_version(), + writer_features: if snapshot.min_writer_version() < 7 { + None + } else { + table.get_writer_features().cloned() + }, + reader_features: if snapshot.min_reader_version() < 3 { + None + } else { + table.get_reader_features().cloned() + }, } } else { - action::Protocol { + Protocol { min_reader_version: max( table.get_min_reader_version(), snapshot.min_reader_version(), @@ -216,6 +226,8 @@ async fn execute( table.get_min_writer_version(), snapshot.min_writer_version(), ), + writer_features: snapshot.writer_features().cloned(), + reader_features: snapshot.reader_features().cloned(), } }; actions.push(Action::protocol(protocol)); @@ -229,6 +241,7 @@ async fn execute( datetime: datetime_to_restore.map(|time| -> i64 { time.timestamp_millis() }), }, &actions, + &snapshot, None, ) .await?; @@ -249,10 +262,10 @@ async fn execute( async fn check_files_available( object_store: &dyn ObjectStore, - files: &Vec, + files: &Vec, ) -> DeltaResult<()> { for file in files { - let file_path = Path::from(file.path.clone()); + let file_path = Path::parse(file.path.clone())?; match object_store.head(&file_path).await { Ok(_) => {} Err(ObjectStoreError::NotFound { .. }) => { diff --git a/rust/src/operations/transaction/conflict_checker.rs b/crates/deltalake-core/src/operations/transaction/conflict_checker.rs similarity index 95% rename from rust/src/operations/transaction/conflict_checker.rs rename to crates/deltalake-core/src/operations/transaction/conflict_checker.rs index 1058400787..6bbc2a9d45 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/crates/deltalake-core/src/operations/transaction/conflict_checker.rs @@ -5,11 +5,11 @@ use std::io::{BufRead, BufReader, Cursor}; use object_store::ObjectStore; use super::CommitInfo; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; -use crate::delta_config::IsolationLevel; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; +use crate::table::config::IsolationLevel; +use crate::table::state::DeltaTableState; #[cfg(feature = "datafusion")] use super::state::AddContainer; @@ -53,8 +53,8 @@ pub enum CommitConflictError { /// you may need to upgrade your Delta Lake version. /// - When multiple writers are creating or replacing a table at the same time. /// - When multiple writers are writing to an empty path at the same time. - #[error("Protocol changed since last commit.")] - ProtocolChanged, + #[error("Protocol changed since last commit: {0}")] + ProtocolChanged(String), /// Error returned when the table requires an unsupported writer version #[error("Delta-rs does not support writer version {0}")] @@ -114,8 +114,11 @@ impl<'a> TransactionInfo<'a> { actions: &'a Vec, read_whole_table: bool, ) -> DeltaResult { + use datafusion::prelude::SessionContext; + + let session = SessionContext::new(); let read_predicates = read_predicates - .map(|pred| read_snapshot.parse_predicate_expression(pred)) + .map(|pred| read_snapshot.parse_predicate_expression(pred, &session.state())) .transpose()?; Ok(Self { txn_id: "".into(), @@ -392,10 +395,18 @@ impl<'a> ConflictChecker<'a> { /// to read and write against the protocol set by the committed transaction. fn check_protocol_compatibility(&self) -> Result<(), CommitConflictError> { for p in self.winning_commit_summary.protocol() { - if self.txn_info.read_snapshot.min_reader_version() < p.min_reader_version - || self.txn_info.read_snapshot.min_writer_version() < p.min_writer_version - { - return Err(CommitConflictError::ProtocolChanged); + let (win_read, curr_read) = ( + p.min_reader_version, + self.txn_info.read_snapshot.min_reader_version(), + ); + let (win_write, curr_write) = ( + p.min_writer_version, + self.txn_info.read_snapshot.min_writer_version(), + ); + if curr_read < win_read || win_write < curr_write { + return Err(CommitConflictError::ProtocolChanged( + format!("reqired read/write {win_read}/{win_write}, current read/write {curr_read}/{curr_write}"), + )); }; } if !self.winning_commit_summary.protocol().is_empty() @@ -405,7 +416,9 @@ impl<'a> ConflictChecker<'a> { .iter() .any(|a| matches!(a, Action::protocol(_))) { - return Err(CommitConflictError::ProtocolChanged); + return Err(CommitConflictError::ProtocolChanged( + "protocol changed".into(), + )); }; Ok(()) } @@ -631,7 +644,7 @@ mod tests { use super::super::test_utils as tu; use super::super::test_utils::init_table_actions; use super::*; - use crate::action::Action; + use crate::protocol::Action; #[cfg(feature = "datafusion")] use datafusion_expr::{col, lit}; use serde_json::json; @@ -678,7 +691,7 @@ mod tests { actions: Vec, read_whole_table: bool, ) -> Result<(), CommitConflictError> { - let setup_actions = setup.unwrap_or_else(init_table_actions); + 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 summary = WinningCommitSummary { @@ -704,7 +717,7 @@ mod tests { // the concurrent transaction deletes a file that the current transaction did NOT read let file_not_read = tu::create_add_action("file_not_read", true, get_stats(1, 10)); let file_read = tu::create_add_action("file_read", true, get_stats(100, 10000)); - let mut setup_actions = init_table_actions(); + let mut setup_actions = init_table_actions(None); setup_actions.push(file_not_read); setup_actions.push(file_read); let result = execute_test( @@ -720,7 +733,7 @@ mod tests { // concurrently add file, that the current transaction would not have read let file_added = tu::create_add_action("file_added", true, get_stats(1, 10)); let file_read = tu::create_add_action("file_read", true, get_stats(100, 10000)); - let mut setup_actions = init_table_actions(); + let mut setup_actions = init_table_actions(None); setup_actions.push(file_read); let result = execute_test( Some(setup_actions), @@ -784,7 +797,7 @@ mod tests { // delete / read // transaction reads a file that is removed by concurrent transaction let file_read = tu::create_add_action("file_read", true, get_stats(1, 10)); - let mut setup_actions = init_table_actions(); + let mut setup_actions = init_table_actions(None); setup_actions.push(file_read); let result = execute_test( Some(setup_actions), @@ -818,7 +831,10 @@ mod tests { vec![tu::create_protocol_action(None, None)], false, ); - assert!(matches!(result, Err(CommitConflictError::ProtocolChanged))); + assert!(matches!( + result, + Err(CommitConflictError::ProtocolChanged(_)) + )); // taint whole table // `read_whole_table` should disallow any concurrent change, even if the change @@ -826,7 +842,7 @@ mod tests { let file_part1 = tu::create_add_action("file_part1", true, get_stats(1, 10)); let file_part2 = tu::create_add_action("file_part2", true, get_stats(11, 100)); let file_part3 = tu::create_add_action("file_part3", true, get_stats(101, 1000)); - let mut setup_actions = init_table_actions(); + let mut setup_actions = init_table_actions(None); setup_actions.push(file_part1); let result = execute_test( Some(setup_actions), @@ -842,7 +858,7 @@ mod tests { // `read_whole_table` should disallow any concurrent remove actions let file_part1 = tu::create_add_action("file_part1", true, get_stats(1, 10)); let file_part2 = tu::create_add_action("file_part2", true, get_stats(11, 100)); - let mut setup_actions = init_table_actions(); + let mut setup_actions = init_table_actions(None); setup_actions.push(file_part1); let result = execute_test( Some(setup_actions), diff --git a/rust/src/operations/transaction/mod.rs b/crates/deltalake-core/src/operations/transaction/mod.rs similarity index 79% rename from rust/src/operations/transaction/mod.rs rename to crates/deltalake-core/src/operations/transaction/mod.rs index fe99af1749..738ae404ec 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/crates/deltalake-core/src/operations/transaction/mod.rs @@ -5,11 +5,11 @@ use object_store::path::Path; use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::{Map, Value}; -use crate::action::{Action, CommitInfo, DeltaOperation}; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, CommitInfo, DeltaOperation}; use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; mod conflict_checker; #[cfg(feature = "datafusion")] @@ -48,6 +48,11 @@ pub enum TransactionError { /// 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, } impl From for DeltaTableError { @@ -68,9 +73,18 @@ impl From for DeltaTableError { // Convert actions to their json representation fn log_entry_from_actions<'a>( actions: impl IntoIterator, + read_snapshot: &DeltaTableState, ) -> Result { + let append_only = read_snapshot.table_config().append_only(); let mut jsons = Vec::::new(); for action in actions { + if append_only { + if let Action::remove(remove) = action { + if remove.data_change { + return Err(TransactionError::DeltaTableAppendOnly); + } + } + } let json = serde_json::to_string(action) .map_err(|e| TransactionError::SerializeLogJson { json_err: e })?; jsons.push(json); @@ -81,6 +95,7 @@ fn log_entry_from_actions<'a>( pub(crate) fn get_commit_bytes( operation: &DeltaOperation, actions: &Vec, + read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> Result { if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { @@ -99,9 +114,13 @@ pub(crate) fn get_commit_bytes( actions .iter() .chain(std::iter::once(&Action::commitInfo(commit_info))), + read_snapshot, )?)) } else { - Ok(bytes::Bytes::from(log_entry_from_actions(actions)?)) + Ok(bytes::Bytes::from(log_entry_from_actions( + actions, + read_snapshot, + )?)) } } @@ -112,10 +131,11 @@ pub(crate) async fn prepare_commit<'a>( storage: &dyn ObjectStore, operation: &DeltaOperation, actions: &Vec, + read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> Result { // Serialize all actions that are part of this log entry. - let log_entry = get_commit_bytes(operation, actions, app_metadata)?; + let log_entry = get_commit_bytes(operation, actions, read_snapshot, app_metadata)?; // 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. @@ -177,7 +197,8 @@ pub async fn commit_with_retries( app_metadata: Option>, max_retries: usize, ) -> DeltaResult { - let tmp_commit = prepare_commit(storage, &operation, actions, app_metadata).await?; + let tmp_commit = + prepare_commit(storage, &operation, actions, read_snapshot, app_metadata).await?; let mut attempt_number = 1; @@ -218,9 +239,11 @@ pub async fn commit_with_retries( #[cfg(all(test, feature = "parquet"))] mod tests { - use self::test_utils::init_table_actions; + use self::test_utils::{create_remove_action, init_table_actions}; use super::*; + use crate::DeltaConfigKey; use object_store::memory::InMemory; + use std::collections::HashMap; #[test] fn test_commit_uri_from_version() { @@ -232,13 +255,36 @@ mod tests { #[test] fn test_log_entry_from_actions() { - let actions = init_table_actions(); - let entry = log_entry_from_actions(&actions).unwrap(); + let actions = init_table_actions(None); + let state = DeltaTableState::from_actions(actions.clone(), 0).unwrap(); + let entry = log_entry_from_actions(&actions, &state).unwrap(); let lines: Vec<_> = entry.lines().collect(); // writes every action to a line assert_eq!(actions.len(), lines.len()) } + fn remove_action_exists_when_delta_table_is_append_only( + data_change: bool, + ) -> Result { + let remove = create_remove_action("test_append_only", data_change); + let mut actions = init_table_actions(Some(HashMap::from([( + DeltaConfigKey::AppendOnly.as_ref().to_string(), + Some("true".to_string()), + )]))); + actions.push(remove); + let state = + DeltaTableState::from_actions(actions.clone(), 0).expect("Failed to get table state"); + log_entry_from_actions(&actions, &state) + } + + #[test] + fn test_remove_action_exists_when_delta_table_is_append_only() { + let _err = remove_action_exists_when_delta_table_is_append_only(true) + .expect_err("Remove action is included when Delta table is append-only. Should error"); + let _actions = remove_action_exists_when_delta_table_is_append_only(false) + .expect("Data is not changed by the Remove action. Should succeed"); + } + #[tokio::test] async fn test_try_commit_transaction() { let store = InMemory::new(); diff --git a/rust/src/operations/transaction/state.rs b/crates/deltalake-core/src/operations/transaction/state.rs similarity index 85% rename from rust/src/operations/transaction/state.rs rename to crates/deltalake-core/src/operations/transaction/state.rs index 98a20d8866..bb9c3ff35e 100644 --- a/rust/src/operations/transaction/state.rs +++ b/crates/deltalake-core/src/operations/transaction/state.rs @@ -5,26 +5,23 @@ 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::config::ConfigOptions; use datafusion_common::scalar::ScalarValue; -use datafusion_common::{Column, DFSchema, Result as DFResult, TableReference}; -use datafusion_expr::{AggregateUDF, Expr, ScalarUDF, TableSource}; -use datafusion_sql::planner::{ContextProvider, SqlToRel}; +use datafusion_common::{Column, DFSchema}; +use datafusion_expr::Expr; use itertools::Either; use object_store::ObjectStore; use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; -use sqlparser::dialect::GenericDialect; -use sqlparser::parser::Parser; -use sqlparser::tokenizer::Tokenizer; -use crate::action::Add; +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, }; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::table_state::DeltaTableState; +use crate::protocol::Add; +use crate::table::state::DeltaTableState; impl DeltaTableState { /// Get the table schema as an [`ArrowSchemaRef`] @@ -104,27 +101,13 @@ impl DeltaTableState { } /// Parse an expression string into a datafusion [`Expr`] - pub fn parse_predicate_expression(&self, expr: impl AsRef) -> DeltaResult { - let dialect = &GenericDialect {}; - let mut tokenizer = Tokenizer::new(dialect, expr.as_ref()); - let tokens = tokenizer - .tokenize() - .map_err(|err| DeltaTableError::GenericError { - source: Box::new(err), - })?; - let sql = Parser::new(dialect) - .with_tokens(tokens) - .parse_expr() - .map_err(|err| DeltaTableError::GenericError { - source: Box::new(err), - })?; - - // TODO should we add the table name as qualifier when available? - let df_schema = DFSchema::try_from_qualified_schema("", self.arrow_schema()?.as_ref())?; - let context_provider = DummyContextProvider::default(); - let sql_to_rel = SqlToRel::new(&context_provider); - - Ok(sql_to_rel.sql_to_expr(sql, &df_schema, &mut Default::default())?) + 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. @@ -342,59 +325,33 @@ impl PruningStatistics for DeltaTableState { } } -#[derive(Default)] -struct DummyContextProvider { - options: ConfigOptions, -} - -impl ContextProvider for DummyContextProvider { - fn get_table_provider(&self, _name: TableReference) -> DFResult> { - unimplemented!() - } - - fn get_function_meta(&self, _name: &str) -> Option> { - unimplemented!() - } - - fn get_aggregate_meta(&self, _name: &str) -> Option> { - unimplemented!() - } - - fn get_variable_type(&self, _: &[String]) -> Option { - unimplemented!() - } - - fn options(&self) -> &ConfigOptions { - &self.options - } - - fn get_window_meta(&self, _name: &str) -> Option> { - unimplemented!() - } -} - #[cfg(test)] mod tests { use super::*; 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 state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); + let snapshot = DeltaTableState::from_actions(init_table_actions(None), 0).unwrap(); + let session = SessionContext::new(); + let state = session.state(); // parses simple expression - let parsed = state.parse_predicate_expression("value > 10").unwrap(); + let parsed = snapshot + .parse_predicate_expression("value > 10", &state) + .unwrap(); let expected = col("value").gt(lit::(10)); assert_eq!(parsed, expected); // fails for unknown column - let parsed = state.parse_predicate_expression("non_existent > 10"); + let parsed = snapshot.parse_predicate_expression("non_existent > 10", &state); assert!(parsed.is_err()); // parses complex expression - let parsed = state - .parse_predicate_expression("value > 10 OR value <= 0") + let parsed = snapshot + .parse_predicate_expression("value > 10 OR value <= 0", &state) .unwrap(); let expected = col("value") .gt(lit::(10)) @@ -404,7 +361,7 @@ mod tests { #[test] fn test_files_matching_predicate() { - let mut actions = init_table_actions(); + let mut actions = init_table_actions(None); actions.push(create_add_action("excluded", true, Some("{\"numRecords\":10,\"minValues\":{\"value\":1},\"maxValues\":{\"value\":10},\"nullCount\":{\"value\":0}}".into()))); 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()))); diff --git a/rust/src/operations/transaction/test_utils.rs b/crates/deltalake-core/src/operations/transaction/test_utils.rs similarity index 87% rename from rust/src/operations/transaction/test_utils.rs rename to crates/deltalake-core/src/operations/transaction/test_utils.rs index 92d981d5e1..e4ae14b2ed 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/crates/deltalake-core/src/operations/transaction/test_utils.rs @@ -1,12 +1,12 @@ #![allow(unused)] -use super::{prepare_commit, try_commit_transaction, CommitInfo}; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; -use crate::table_state::DeltaTableState; -use crate::{ - DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, -}; use std::collections::HashMap; +use super::{prepare_commit, try_commit_transaction, CommitInfo}; +use crate::protocol::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; +use crate::table::state::DeltaTableState; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; + pub fn create_add_action( path: impl Into, data_change: bool, @@ -33,6 +33,8 @@ pub fn create_protocol_action(max_reader: Option, max_writer: Option) let protocol = Protocol { min_reader_version: max_reader.unwrap_or(crate::operations::MAX_SUPPORTED_READER_VERSION), min_writer_version: max_writer.unwrap_or(crate::operations::MAX_SUPPORTED_WRITER_VERSION), + writer_features: None, + reader_features: None, }; Action::protocol(protocol) } @@ -72,7 +74,7 @@ pub fn create_metadata_action( Action::metaData(MetaData::try_from(metadata).unwrap()) } -pub fn init_table_actions() -> Vec { +pub fn init_table_actions(configuration: Option>>) -> Vec { let raw = r#" { "timestamp": 1670892998177, @@ -96,7 +98,7 @@ pub fn init_table_actions() -> Vec { vec![ Action::commitInfo(commit_info), create_protocol_action(None, None), - create_metadata_action(None, None), + create_metadata_action(None, configuration), ] } @@ -127,13 +129,15 @@ pub async fn create_initialized_table( HashMap::new(), ), ]); - let state = DeltaTableState::from_actions(init_table_actions(), 0).unwrap(); + let state = DeltaTableState::from_actions(init_table_actions(None), 0).unwrap(); let operation = DeltaOperation::Create { mode: SaveMode::ErrorIfExists, location: "location".into(), protocol: Protocol { min_reader_version: 1, min_writer_version: 1, + writer_features: None, + reader_features: None, }, metadata: DeltaTableMetaData::new( None, @@ -144,8 +148,8 @@ pub async fn create_initialized_table( configuration.unwrap_or_default(), ), }; - let actions = init_table_actions(); - let prepared_commit = prepare_commit(storage.as_ref(), &operation, &actions, None) + let actions = init_table_actions(None); + let prepared_commit = prepare_commit(storage.as_ref(), &operation, &actions, &state, None) .await .unwrap(); try_commit_transaction(storage.as_ref(), &prepared_commit, 0) diff --git a/rust/src/operations/update.rs b/crates/deltalake-core/src/operations/update.rs similarity index 93% rename from rust/src/operations/update.rs rename to crates/deltalake-core/src/operations/update.rs index a104732a8f..4764736eb2 100644 --- a/rust/src/operations/update.rs +++ b/crates/deltalake-core/src/operations/update.rs @@ -40,13 +40,14 @@ use datafusion_physical_expr::{ }; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; +use serde::Serialize; use serde_json::{Map, Value}; use crate::{ - action::{Action, DeltaOperation, Remove}, - delta_datafusion::{find_files, parquet_scan_from_actions, register_store}, + delta_datafusion::{expr::fmt_expr_to_sql, find_files, register_store, DeltaScanBuilder}, + protocol::{Action, DeltaOperation, Remove}, storage::{DeltaObjectStore, ObjectStoreRef}, - table_state::DeltaTableState, + table::state::DeltaTableState, DeltaResult, DeltaTable, DeltaTableError, }; @@ -78,7 +79,7 @@ pub struct UpdateBuilder { safe_cast: bool, } -#[derive(Default)] +#[derive(Default, Serialize, Debug)] /// Metrics collected during the Update operation pub struct UpdateMetrics { /// Number of files added. @@ -194,7 +195,7 @@ async fn execute( let predicate = match predicate { Some(predicate) => match predicate { Expression::DataFusion(expr) => Some(expr), - Expression::String(s) => Some(snapshot.parse_predicate_expression(s)?), + Expression::String(s) => Some(snapshot.parse_predicate_expression(s, &state)?), }, None => None, }; @@ -203,7 +204,9 @@ async fn execute( .into_iter() .map(|(key, expr)| match expr { Expression::DataFusion(e) => Ok((key, e)), - Expression::String(s) => snapshot.parse_predicate_expression(s).map(|e| (key, e)), + Expression::String(s) => snapshot + .parse_predicate_expression(s, &state) + .map(|e| (key, e)), }) .collect::, _>>()?; @@ -211,17 +214,9 @@ async fn execute( .current_metadata() .ok_or(DeltaTableError::NoMetadata)?; let table_partition_cols = current_metadata.partition_columns.clone(); - let schema = snapshot.arrow_schema()?; let scan_start = Instant::now(); - let candidates = find_files( - snapshot, - object_store.clone(), - schema.clone(), - &state, - predicate.clone(), - ) - .await?; + let candidates = find_files(snapshot, object_store.clone(), &state, predicate.clone()).await?; metrics.scan_time_ms = Instant::now().duration_since(scan_start).as_millis() as u64; if candidates.candidates.is_empty() { @@ -233,17 +228,11 @@ async fn execute( let execution_props = state.execution_props(); // For each rewrite evaluate the predicate and then modify each expression // to either compute the new value or obtain the old one then write these batches - let parquet_scan = parquet_scan_from_actions( - snapshot, - object_store.clone(), - &candidates.candidates, - &schema, - None, - &state, - None, - None, - ) - .await?; + let scan = DeltaScanBuilder::new(snapshot, object_store.clone(), &state) + .with_files(&candidates.candidates) + .build() + .await?; + let scan = Arc::new(scan); // Create a projection for a new column with the predicate evaluated let input_schema = snapshot.input_schema()?; @@ -262,7 +251,7 @@ async fn execute( let input_dfschema: DFSchema = input_schema.as_ref().clone().try_into()?; let mut expressions: Vec<(Arc, String)> = Vec::new(); - let scan_schema = parquet_scan.schema(); + let scan_schema = scan.schema(); for (i, field) in scan_schema.fields().into_iter().enumerate() { expressions.push(( Arc::new(expressions::Column::new(field.name(), i)), @@ -286,7 +275,7 @@ async fn execute( expressions.push((predicate_expr, "__delta_rs_update_predicate".to_string())); let projection_predicate: Arc = - Arc::new(ProjectionExec::try_new(expressions, parquet_scan)?); + Arc::new(ProjectionExec::try_new(expressions, scan)?); let count_plan = Arc::new(MetricObserverExec::new( projection_predicate.clone(), @@ -416,7 +405,7 @@ async fn execute( metrics.execution_time_ms = Instant::now().duration_since(exec_start).as_millis() as u64; let operation = DeltaOperation::Update { - predicate: Some(predicate.canonical_name()), + predicate: Some(fmt_expr_to_sql(&predicate)?), }; version = commit( object_store.as_ref(), @@ -470,17 +459,20 @@ impl std::future::IntoFuture for UpdateBuilder { #[cfg(test)] mod tests { - use crate::operations::DeltaOps; use crate::writer::test_utils::datafusion::get_data; - use crate::writer::test_utils::{get_arrow_schema, get_delta_schema}; + use crate::writer::test_utils::{ + get_arrow_schema, get_delta_schema, get_record_batch, setup_table_with_configuration, + write_batch, + }; + use crate::DeltaConfigKey; use crate::DeltaTable; - use crate::{action::*, DeltaResult}; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; use arrow_array::Int32Array; use datafusion::assert_batches_sorted_eq; use datafusion::prelude::*; + use serde_json::json; use std::sync::Arc; async fn setup_table(partitions: Option>) -> DeltaTable { @@ -496,13 +488,6 @@ mod tests { table } - async fn write_batch(table: DeltaTable, batch: RecordBatch) -> DeltaResult { - DeltaOps(table) - .write(vec![batch.clone()]) - .with_save_mode(SaveMode::Append) - .await - } - async fn prepare_values_table() -> DeltaTable { let schema = Arc::new(Schema::new(vec![Field::new( "value", @@ -525,6 +510,19 @@ mod tests { DeltaOps::new_in_memory().write(vec![batch]).await.unwrap() } + #[tokio::test] + async fn test_update_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).await; + let _err = DeltaOps(table) + .update() + .with_update("modified", lit("2023-05-14")) + .await + .expect_err("Remove action is included when Delta table is append-only. Should error"); + } + #[tokio::test] async fn test_update_no_predicate() { let schema = get_arrow_schema(&None); @@ -545,7 +543,7 @@ mod tests { ) .unwrap(); - let table = write_batch(table, batch).await.unwrap(); + let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); assert_eq!(table.get_file_uris().count(), 1); @@ -599,11 +597,11 @@ mod tests { // Update a partitioned table where the predicate contains only partition column // The expectation is that a physical scan of data is not required - let table = write_batch(table, batch).await.unwrap(); + let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); assert_eq!(table.get_file_uris().count(), 1); - let (table, metrics) = DeltaOps(table) + let (mut table, metrics) = DeltaOps(table) .update() .with_predicate(col("modified").eq(lit("2021-02-03"))) .with_update("modified", lit("2023-05-14")) @@ -617,6 +615,11 @@ mod tests { 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 parameters = last_commit.operation_parameters.clone().unwrap(); + assert_eq!(parameters["predicate"], json!("modified = '2021-02-03'")); + let expected = vec![ "+----+-------+------------+", "| id | value | modified |", @@ -651,7 +654,7 @@ mod tests { ) .unwrap(); - let table = write_batch(table, batch.clone()).await.unwrap(); + let table = write_batch(table, batch.clone()).await; assert_eq!(table.version(), 1); assert_eq!(table.get_file_uris().count(), 2); @@ -686,7 +689,7 @@ mod tests { // Update a partitioned table where the predicate contains a partition column and non-partition column let table = setup_table(Some(vec!["modified"])).await; - let table = write_batch(table, batch).await.unwrap(); + let table = write_batch(table, batch).await; assert_eq!(table.version(), 1); assert_eq!(table.get_file_uris().count(), 2); diff --git a/rust/src/operations/vacuum.rs b/crates/deltalake-core/src/operations/vacuum.rs similarity index 73% rename from rust/src/operations/vacuum.rs rename to crates/deltalake-core/src/operations/vacuum.rs index 40e7657edf..684e6f6d0a 100644 --- a/rust/src/operations/vacuum.rs +++ b/crates/deltalake-core/src/operations/vacuum.rs @@ -30,10 +30,15 @@ use futures::future::BoxFuture; use futures::{StreamExt, TryStreamExt}; use object_store::Error; use object_store::{path::Path, ObjectStore}; +use serde::Serialize; +use serde_json::{Map, Value}; +use super::transaction::commit; +use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, DeltaOperation}; // Txn CommitInfo use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; /// Errors that can occur during vacuum @@ -96,6 +101,26 @@ pub struct VacuumMetrics { pub files_deleted: Vec, } +/// Details for the Vacuum start operation for the transaction log +#[derive(Serialize)] +#[serde(rename_all = "camelCase")] +pub struct VacuumStartOperationMetrics { + /// The number of files that will be deleted + pub num_files_to_delete: i64, + /// Size of the data to be deleted in bytes + pub size_of_data_to_delete: i64, +} + +/// Details for the Vacuum End operation for the transaction log +#[derive(Serialize)] +#[serde(rename_all = "camelCase")] +pub struct VacuumEndOperationMetrics { + /// The number of actually deleted files + pub num_deleted_files: i64, + /// The number of actually vacuumed directories + pub num_vacuumed_directories: i64, +} + /// Methods to specify various vacuum options and to execute the operation impl VacuumBuilder { /// Create a new [`VacuumBuilder`] @@ -157,6 +182,7 @@ impl VacuumBuilder { let valid_files = self.snapshot.file_paths_iter().collect::>(); let mut files_to_delete = vec![]; + let mut file_sizes = vec![]; let mut all_files = self.store.list(None).await.map_err(DeltaTableError::from)?; let partition_columns = &self @@ -176,9 +202,16 @@ impl VacuumBuilder { } files_to_delete.push(obj_meta.location); + file_sizes.push(obj_meta.size as i64); } - Ok(VacuumPlan { files_to_delete }) + Ok(VacuumPlan { + files_to_delete, + file_sizes, + retention_check_enabled: enforce_retention_duration, + default_retention_millis: min_retention.num_milliseconds(), + specified_retention_millis: Some(retention_period.num_milliseconds()), + }) } } @@ -201,7 +234,7 @@ impl std::future::IntoFuture for VacuumBuilder { )); } - let metrics = plan.execute(&this.store).await?; + let metrics = plan.execute(&this.store, &this.snapshot).await?; Ok(( DeltaTable::new_with_state(this.store, this.snapshot), metrics, @@ -214,11 +247,23 @@ impl std::future::IntoFuture for VacuumBuilder { struct VacuumPlan { /// What files are to be deleted pub files_to_delete: Vec, + /// Size of each file which to delete + pub file_sizes: Vec, + /// If retention check is enabled + pub retention_check_enabled: bool, + /// Default retention in milliseconds + pub default_retention_millis: i64, + /// Overrided retention in milliseconds + pub specified_retention_millis: Option, } impl VacuumPlan { /// Execute the vacuum plan and delete files from underlying storage - pub async fn execute(self, store: &DeltaObjectStore) -> Result { + pub async fn execute( + self, + store: &DeltaObjectStore, + snapshot: &DeltaTableState, + ) -> Result { if self.files_to_delete.is_empty() { return Ok(VacuumMetrics { dry_run: false, @@ -226,6 +271,40 @@ impl VacuumPlan { }); } + let start_operation = DeltaOperation::VacuumStart { + retention_check_enabled: self.retention_check_enabled, + specified_retention_millis: self.specified_retention_millis, + default_retention_millis: self.default_retention_millis, + }; + + let end_operation = DeltaOperation::VacuumEnd { + 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 { + 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 = Map::::new(); + + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( + "clientVersion".to_string(), + Value::String(format!("delta-rs.{}", crate_version())), + ); + 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?; + // Finish VACUUM START COMMIT + let locations = futures::stream::iter(self.files_to_delete) .map(Result::Ok) .boxed(); @@ -240,6 +319,31 @@ impl VacuumPlan { .try_collect::>() .await?; + // Create end metadata + let end_metrics = serde_json::to_value(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 = Map::::new(); + + commit_info.timestamp = Some(Utc::now().timestamp_millis()); + extra_info.insert( + "clientVersion".to_string(), + Value::String(format!("delta-rs.{}", crate_version())), + ); + 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?; + // Finish VACUUM END COMMIT + Ok(VacuumMetrics { files_deleted, dry_run: false, diff --git a/rust/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs similarity index 91% rename from rust/src/operations/write.rs rename to crates/deltalake-core/src/operations/write.rs index 41dfeaae5a..7472d02d07 100644 --- a/rust/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -14,6 +14,16 @@ //! //! 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::sync::Arc; @@ -32,15 +42,15 @@ use serde_json::Map; use super::writer::{DeltaWriter, WriterConfig}; use super::MAX_SUPPORTED_WRITER_VERSION; use super::{transaction::commit, CreateBuilder}; -use crate::action::{Action, Add, DeltaOperation, Remove, SaveMode}; -use crate::delta::DeltaTable; use crate::delta_datafusion::DeltaDataChecker; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, Add, DeltaOperation, Remove, SaveMode}; use crate::schema::Schema; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::writer::record_batch::divide_by_partition_values; use crate::writer::utils::PartitionPath; +use crate::DeltaTable; #[derive(thiserror::Error, Debug)] enum WriteError { @@ -552,19 +562,35 @@ fn cast_record_batch( #[cfg(test)] mod tests { use super::*; - use crate::operations::DeltaOps; + use crate::operations::{collect_sendable_stream, DeltaOps}; + use crate::protocol::SaveMode; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::{ get_delta_schema, get_delta_schema_with_nested_struct, get_record_batch, - get_record_batch_with_nested_struct, + get_record_batch_with_nested_struct, setup_table_with_configuration, write_batch, }; + 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::assert_batches_sorted_eq; + 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(); @@ -855,4 +881,47 @@ mod tests { &expected ); } + + #[tokio::test] + async fn test_special_characters_write_read() { + let tmp_dir = tempdir::TempDir::new("test").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 |", + "+------+-----------------------------------+", + r#"| test | $%&/()=^"[]#*?.:_- {=}|`<>~/\r\n+ |"#, + "+------+-----------------------------------+", + ]; + + assert_batches_eq!(&expected, &data); + } } diff --git a/rust/src/operations/writer.rs b/crates/deltalake-core/src/operations/writer.rs similarity index 98% rename from rust/src/operations/writer.rs rename to crates/deltalake-core/src/operations/writer.rs index 4fef892bf8..05bda44ae6 100644 --- a/rust/src/operations/writer.rs +++ b/crates/deltalake-core/src/operations/writer.rs @@ -2,16 +2,6 @@ use std::collections::HashMap; -use crate::action::Add; -use crate::storage::ObjectStoreRef; -use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; -use crate::writer::stats::create_add; -use crate::writer::utils::{ - self, arrow_schema_without_partitions, record_batch_without_partitions, PartitionPath, - ShareableBuffer, -}; -use crate::{crate_version, DeltaResult, DeltaTableError}; - use arrow::datatypes::SchemaRef as ArrowSchemaRef; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; @@ -21,6 +11,17 @@ use parquet::arrow::ArrowWriter; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; +use crate::crate_version; +use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; +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, +}; + // TODO databricks often suggests a file size of 100mb, should we set this default? const DEFAULT_TARGET_FILE_SIZE: usize = 104_857_600; const DEFAULT_WRITE_BATCH_SIZE: usize = 1024; @@ -247,7 +248,7 @@ impl PartitionWriterConfig { .map_err(|err| WriteError::FileName { source: Box::new(err), })?; - let prefix = Path::from(part_path.as_ref()); + let prefix = Path::parse(part_path.as_ref())?; let writer_properties = writer_properties.unwrap_or_else(|| { WriterProperties::builder() .set_created_by(format!("delta-rs version {}", crate_version())) @@ -304,7 +305,7 @@ impl PartitionWriter { fn next_data_path(&mut self) -> Path { self.part_counter += 1; - utils::next_data_path( + next_data_path( &self.config.prefix, self.part_counter, &self.writer_id, diff --git a/rust/src/action/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs similarity index 83% rename from rust/src/action/checkpoints.rs rename to crates/deltalake-core/src/protocol/checkpoints.rs index 86dbc7bc87..5667b4e1b2 100644 --- a/rust/src/action/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -3,28 +3,28 @@ use std::collections::HashMap; use std::convert::TryFrom; use std::iter::Iterator; -use std::ops::Add; use arrow::datatypes::Schema as ArrowSchema; use arrow::error::ArrowError; use arrow::json::ReaderBuilder; -use chrono::{DateTime, Datelike, Duration, Utc}; -use futures::StreamExt; +use chrono::{Datelike, Utc}; +use futures::{StreamExt, TryStreamExt}; use lazy_static::lazy_static; use log::*; -use object_store::{path::Path, ObjectMeta, ObjectStore}; +use object_store::ObjectStore; use parquet::arrow::ArrowWriter; use parquet::errors::ParquetError; use regex::Regex; use serde_json::Value; -use super::{Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; -use crate::delta_arrow::delta_log_schema_for_table; +use super::{time_utils, Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; +use crate::arrow_convert::delta_log_schema_for_table; +use crate::schema::*; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; -use crate::{open_table_with_version, time_utils, CheckPoint, DeltaTable}; -use crate::{schema::*, CheckPointBuilder}; +use crate::table::state::DeltaTableState; +use crate::table::{CheckPoint, CheckPointBuilder}; +use crate::{open_table_with_version, DeltaTable}; type SchemaPath = Vec; @@ -74,7 +74,7 @@ pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), ProtocolError> /// Delete expires log files before given version from table. The table log retention is based on /// the `logRetentionDuration` property of the Delta Table, 30 days by default. -pub async fn cleanup_metadata(table: &DeltaTable) -> Result { +pub async fn cleanup_metadata(table: &DeltaTable) -> Result { let log_retention_timestamp = Utc::now().timestamp_millis() - table.get_state().log_retention_millis(); cleanup_expired_logs_for( @@ -140,148 +140,57 @@ pub async fn create_checkpoint_for( Ok(()) } -async fn flush_delete_files bool>( - storage: &DeltaObjectStore, - maybe_delete_files: &mut Vec<(i64, ObjectMeta)>, - files_to_delete: &mut Vec<(i64, ObjectMeta)>, - should_delete_file: T, -) -> Result { - if !maybe_delete_files.is_empty() && should_delete_file(maybe_delete_files.last().unwrap()) { - files_to_delete.append(maybe_delete_files); - } - - let deleted = files_to_delete - .iter_mut() - .map(|file| async move { - match storage.delete(&file.1.location).await { - Ok(_) => Ok(1), - Err(e) => Err(ProtocolError::from(e)), - } - }) - .collect::>(); - - let mut deleted_num = 0; - for x in deleted { - match x.await { - Ok(_) => deleted_num += 1, - Err(e) => return Err(e), - } - } - - files_to_delete.clear(); - - Ok(deleted_num) -} - -/// exposed only for integration testing - DO NOT USE otherwise +/// Deletes all delta log commits that are older than the cutoff time +/// and less than the specified version. pub async fn cleanup_expired_logs_for( until_version: i64, storage: &DeltaObjectStore, - log_retention_timestamp: i64, -) -> Result { + cutoff_timestamp: i64, +) -> Result { lazy_static! { static ref DELTA_LOG_REGEX: Regex = Regex::new(r"_delta_log/(\d{20})\.(json|checkpoint).*$").unwrap(); } - let mut deleted_log_num = 0; - - // Get file objects from table. - let mut candidates: Vec<(i64, ObjectMeta)> = Vec::new(); - let mut stream = storage.list(Some(storage.log_path())).await?; - while let Some(obj_meta) = stream.next().await { - let obj_meta = obj_meta?; - - let ts = obj_meta.last_modified.timestamp_millis(); - - 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 < until_version && ts <= log_retention_timestamp { - candidates.push((log_ver, obj_meta)); - } - } - } - - // Sort files by file object version. - candidates.sort_by(|a, b| a.0.cmp(&b.0)); - - let mut last_file: (i64, ObjectMeta) = ( - 0, - ObjectMeta { - location: Path::from(""), - last_modified: DateTime::::MIN_UTC, - size: 0, - e_tag: None, - }, - ); - let file_needs_time_adjustment = - |current_file: &(i64, ObjectMeta), last_file: &(i64, ObjectMeta)| { - last_file.0 < current_file.0 - && last_file.1.last_modified.timestamp() >= current_file.1.last_modified.timestamp() - }; - - let should_delete_file = |file: &(i64, ObjectMeta)| { - file.1.last_modified.timestamp() <= log_retention_timestamp && file.0 < until_version - }; - - let mut maybe_delete_files: Vec<(i64, ObjectMeta)> = Vec::new(); - let mut files_to_delete: Vec<(i64, ObjectMeta)> = Vec::new(); - - // Init - if !candidates.is_empty() { - let removed = candidates.remove(0); - last_file = (removed.0, removed.1.clone()); - maybe_delete_files.push(removed); - } - - let mut current_file: (i64, ObjectMeta); - loop { - if candidates.is_empty() { - deleted_log_num += flush_delete_files( - storage, - &mut maybe_delete_files, - &mut files_to_delete, - should_delete_file, - ) - .await?; - - return Ok(deleted_log_num); - } - current_file = candidates.remove(0); - - if file_needs_time_adjustment(¤t_file, &last_file) { - let updated = ( - current_file.0, - ObjectMeta { - location: current_file.1.location.clone(), - last_modified: last_file.1.last_modified.add(Duration::seconds(1)), - size: 0, - e_tag: None, - }, - ); - maybe_delete_files.push(updated); - last_file = ( - maybe_delete_files.last().unwrap().0, - maybe_delete_files.last().unwrap().1.clone(), - ); - } else { - let deleted = flush_delete_files( - storage, - &mut maybe_delete_files, - &mut files_to_delete, - should_delete_file, - ) - .await?; - if deleted == 0 { - return Ok(deleted_log_num); - } - deleted_log_num += deleted; + // 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. + let deleted = storage + .delete_stream( + storage + .list(Some(storage.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 { + if meta.is_err() { + error!("Error received while cleaning up expired logs: {:?}", meta); + return None; + } + let meta = meta.unwrap(); + let ts = meta.last_modified.timestamp_millis(); + + match DELTA_LOG_REGEX.captures(meta.location.as_ref()) { + Some(captures) => { + let log_ver_str = captures.get(1).unwrap().as_str(); + let log_ver: i64 = log_ver_str.parse().unwrap(); + if log_ver < until_version && ts <= cutoff_timestamp { + // This location is ready to be deleted + Some(Ok(meta.location)) + } else { + None + } + } + None => None, + } + }) + .boxed(), + ) + .try_collect::>() + .await?; - maybe_delete_files.push(current_file.clone()); - last_file = current_file; - } - } + debug!("Deleted {} expired logs", deleted.len()); + Ok(deleted.len()) } fn parquet_bytes_from_state( @@ -320,6 +229,8 @@ fn parquet_bytes_from_state( let jsons = std::iter::once(Action::protocol(Protocol { min_reader_version: state.min_reader_version(), min_writer_version: state.min_writer_version(), + writer_features: None, + reader_features: None, })) // metaData .chain(std::iter::once(Action::metaData(MetaData::try_from( diff --git a/rust/src/action/mod.rs b/crates/deltalake-core/src/protocol/mod.rs similarity index 87% rename from rust/src/action/mod.rs rename to crates/deltalake-core/src/protocol/mod.rs index 5547129550..66f06b13a1 100644 --- a/rust/src/action/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -8,6 +8,8 @@ pub mod checkpoints; pub mod parquet2_read; #[cfg(feature = "parquet")] mod parquet_read; +mod serde_path; +mod time_utils; #[cfg(feature = "arrow")] use arrow_schema::ArrowError; @@ -15,20 +17,20 @@ use futures::StreamExt; use lazy_static::lazy_static; use log::*; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; -use percent_encoding::percent_decode; use regex::Regex; use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; use std::borrow::Borrow; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::hash::{Hash, Hasher}; use std::mem::take; use std::str::FromStr; -use crate::delta_config::IsolationLevel; use crate::errors::DeltaResult; use crate::storage::ObjectStoreRef; -use crate::{delta::CheckPoint, schema::*, DeltaTableMetaData}; +use crate::table::config::IsolationLevel; +use crate::table::DeltaTableMetaData; +use crate::{schema::*, table::CheckPoint}; /// Error returned when an invalid Delta log action is encountered. #[allow(missing_docs)] @@ -105,13 +107,6 @@ pub enum ProtocolError { }, } -fn decode_path(raw_path: &str) -> Result { - percent_decode(raw_path.as_bytes()) - .decode_utf8() - .map(|c| c.to_string()) - .map_err(|e| ProtocolError::InvalidField(format!("Decode path failed for action: {e}"))) -} - /// Struct used to represent minValues and maxValues in add action statistics. #[derive(Serialize, Deserialize, Debug, PartialEq, Eq)] #[serde(untagged)] @@ -255,6 +250,7 @@ pub struct StatsParsed { pub struct AddCDCFile { /// A relative path, from the root of the table, or an /// absolute path to a CDC file + #[serde(with = "serde_path")] pub path: String, /// The size of this file in bytes pub size: i64, @@ -351,6 +347,7 @@ impl Eq for DeletionVector {} #[serde(rename_all = "camelCase")] pub struct Add { /// A relative path, from the root of the table, to a file that should be added to the table + #[serde(with = "serde_path")] pub path: String, /// The size of this file in bytes pub size: i64, @@ -403,9 +400,11 @@ pub struct Add { #[serde(skip_serializing, skip_deserializing)] pub stats_parsed: Option, /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, - ///Metadata about deletion vector + /// Metadata about deletion vector + #[serde(skip_serializing_if = "Option::is_none")] pub deletion_vector: Option, } @@ -431,11 +430,6 @@ impl PartialEq for Add { impl Eq for Add {} impl Add { - /// Returns the Add action with path decoded. - pub fn path_decoded(self) -> Result { - decode_path(&self.path).map(|path| Self { path, ..self }) - } - /// 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> { @@ -506,6 +500,12 @@ impl Default for Format { } } +/// Return a default empty schema to be used for edge-cases when a schema is missing +fn default_schema() -> String { + warn!("A `metaData` action was missing a `schemaString` and has been given an empty schema"); + r#"{"type":"struct", "fields": []}"#.into() +} + /// Action that describes the metadata of the table. /// This is a top-level action in Delta log entries. #[derive(Serialize, Deserialize, Debug, Default, Clone)] @@ -520,6 +520,7 @@ pub struct MetaData { /// Specification of the encoding for the files stored in the table pub format: Format, /// Schema of the table + #[serde(default = "default_schema")] pub schema_string: String, /// An array containing the names of columns by which the data should be partitioned pub partition_columns: Vec, @@ -562,6 +563,7 @@ impl TryFrom for MetaData { #[serde(rename_all = "camelCase")] pub struct Remove { /// The path of the file that is removed from the table. + #[serde(with = "serde_path")] pub path: String, /// The timestamp when the remove was added to table state. pub deletion_timestamp: Option, @@ -574,12 +576,16 @@ pub struct Remove { /// it's still nullable so we keep it as Option<> for compatibly. pub extended_file_metadata: Option, /// A map from partition column to value for this file. + #[serde(skip_serializing_if = "Option::is_none")] pub partition_values: Option>>, /// Size of this file in bytes + #[serde(skip_serializing_if = "Option::is_none")] pub size: Option, /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, - ///Metadata about deletion vector + /// Metadata about deletion vector + #[serde(skip_serializing_if = "Option::is_none")] pub deletion_vector: Option, } @@ -610,13 +616,6 @@ impl PartialEq for Remove { } } -impl Remove { - /// Returns the Remove action with path decoded. - pub fn path_decoded(self) -> Result { - decode_path(&self.path).map(|path| Self { path, ..self }) - } -} - /// Action used by streaming systems to track progress using application-specific versions to /// enable idempotency. #[derive(Serialize, Deserialize, Debug, Default, Clone)] @@ -641,6 +640,190 @@ pub struct Protocol { /// Minimum version of the Delta write protocol a client must implement to correctly read the /// table. pub min_writer_version: i32, + /// Table features are missing from older versions + /// The table features this reader supports + #[serde(skip_serializing_if = "Option::is_none")] + pub reader_features: Option>, + /// Table features are missing from older versions + /// The table features this writer supports + #[serde(skip_serializing_if = "Option::is_none")] + pub writer_features: Option>, +} + +/// Features table readers can support as well as let users know +/// what is supported +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, Hash)] +pub enum ReaderFeatures { + /// Mapping of one column to another + #[serde(alias = "columnMapping")] + COLUMN_MAPPING, + /// Deletion vectors for merge, update, delete + #[serde(alias = "deletionVectors")] + DELETION_VECTORS, + /// timestamps without timezone support + #[serde(alias = "timestampNtz")] + TIMESTAMP_WITHOUT_TIMEZONE, + /// version 2 of checkpointing + #[serde(alias = "v2Checkpoint")] + V2_CHECKPOINT, + /// If we do not match any other reader features + #[serde(untagged)] + OTHER(String), +} + +#[allow(clippy::from_over_into)] +impl Into for ReaderFeatures { + fn into(self) -> usize { + match self { + ReaderFeatures::OTHER(_) => 0, + ReaderFeatures::COLUMN_MAPPING => 2, + ReaderFeatures::DELETION_VECTORS + | ReaderFeatures::TIMESTAMP_WITHOUT_TIMEZONE + | ReaderFeatures::V2_CHECKPOINT => 3, + } + } +} + +#[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::COLUMN_MAPPING, + "deletionVectors" => ReaderFeatures::DELETION_VECTORS, + "timestampNtz" => ReaderFeatures::TIMESTAMP_WITHOUT_TIMEZONE, + "v2Checkpoint" => ReaderFeatures::V2_CHECKPOINT, + f => ReaderFeatures::OTHER(f.to_string()), + }, + f => ReaderFeatures::OTHER(f.to_string()), + } + } +} + +impl From for ReaderFeatures { + fn from(value: String) -> Self { + match value.as_str() { + "columnMapping" => ReaderFeatures::COLUMN_MAPPING, + "deletionVectors" => ReaderFeatures::DELETION_VECTORS, + "timestampNtz" => ReaderFeatures::TIMESTAMP_WITHOUT_TIMEZONE, + "v2Checkpoint" => ReaderFeatures::V2_CHECKPOINT, + f => ReaderFeatures::OTHER(f.to_string()), + } + } +} + +/// Features table writers can support as well as let users know +/// what is supported +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, Hash)] +pub enum WriterFeatures { + /// Append Only Tables + #[serde(alias = "appendOnly")] + APPEND_ONLY, + /// Table invariants + #[serde(alias = "invariants")] + INVARIANTS, + /// Check constraints on columns + #[serde(alias = "checkConstraints")] + CHECK_CONSTRAINTS, + /// CDF on a table + #[serde(alias = "changeDataFeed")] + CHANGE_DATA_FEED, + /// Columns with generated values + #[serde(alias = "generatedColumns")] + GENERATED_COLUMNS, + /// Mapping of one column to another + #[serde(alias = "columnMapping")] + COLUMN_MAPPING, + /// ID Columns + #[serde(alias = "identityColumns")] + IDENTITY_COLUMNS, + /// Deletion vectors for merge, update, delete + #[serde(alias = "deletionVectors")] + DELETION_VECTORS, + /// Row tracking on tables + #[serde(alias = "rowTracking")] + ROW_TRACKING, + /// timestamps without timezone support + #[serde(alias = "timestampNtz")] + TIMESTAMP_WITHOUT_TIMEZONE, + /// domain specific metadata + #[serde(alias = "domainMetadata")] + DOMAIN_METADATA, + /// version 2 of checkpointing + #[serde(alias = "v2Checkpoint")] + V2_CHECKPOINT, + /// Iceberg compatability support + #[serde(alias = "icebergCompatV1")] + ICEBERG_COMPAT_V1, + /// If we do not match any other reader features + #[serde(untagged)] + OTHER(String), +} + +#[allow(clippy::from_over_into)] +impl Into for WriterFeatures { + fn into(self) -> usize { + match self { + WriterFeatures::OTHER(_) => 0, + WriterFeatures::APPEND_ONLY | WriterFeatures::INVARIANTS => 2, + WriterFeatures::CHECK_CONSTRAINTS => 3, + WriterFeatures::CHANGE_DATA_FEED | WriterFeatures::GENERATED_COLUMNS => 4, + WriterFeatures::COLUMN_MAPPING => 5, + WriterFeatures::IDENTITY_COLUMNS + | WriterFeatures::DELETION_VECTORS + | WriterFeatures::ROW_TRACKING + | WriterFeatures::TIMESTAMP_WITHOUT_TIMEZONE + | WriterFeatures::DOMAIN_METADATA + | WriterFeatures::V2_CHECKPOINT + | WriterFeatures::ICEBERG_COMPAT_V1 => 7, + } + } +} + +impl From for WriterFeatures { + fn from(value: String) -> Self { + match value.as_str() { + "appendOnly" => WriterFeatures::APPEND_ONLY, + "invariants" => WriterFeatures::INVARIANTS, + "checkConstraints" => WriterFeatures::CHECK_CONSTRAINTS, + "changeDataFeed" => WriterFeatures::CHANGE_DATA_FEED, + "generatedColumns" => WriterFeatures::GENERATED_COLUMNS, + "columnMapping" => WriterFeatures::COLUMN_MAPPING, + "identityColumns" => WriterFeatures::IDENTITY_COLUMNS, + "deletionVectors" => WriterFeatures::DELETION_VECTORS, + "rowTracking" => WriterFeatures::ROW_TRACKING, + "timestampNtz" => WriterFeatures::TIMESTAMP_WITHOUT_TIMEZONE, + "domainMetadata" => WriterFeatures::DOMAIN_METADATA, + "v2Checkpoint" => WriterFeatures::V2_CHECKPOINT, + "icebergCompatV1" => WriterFeatures::ICEBERG_COMPAT_V1, + f => WriterFeatures::OTHER(f.to_string()), + } + } +} + +#[cfg(all(not(feature = "parquet2"), feature = "parquet"))] +impl From<&parquet::record::Field> for WriterFeatures { + fn from(value: &parquet::record::Field) -> Self { + match value { + parquet::record::Field::Str(feature) => match feature.as_str() { + "appendOnly" => WriterFeatures::APPEND_ONLY, + "invariants" => WriterFeatures::INVARIANTS, + "checkConstraints" => WriterFeatures::CHECK_CONSTRAINTS, + "changeDataFeed" => WriterFeatures::CHANGE_DATA_FEED, + "generatedColumns" => WriterFeatures::GENERATED_COLUMNS, + "columnMapping" => WriterFeatures::COLUMN_MAPPING, + "identityColumns" => WriterFeatures::IDENTITY_COLUMNS, + "deletionVectors" => WriterFeatures::DELETION_VECTORS, + "rowTracking" => WriterFeatures::ROW_TRACKING, + "timestampNtz" => WriterFeatures::TIMESTAMP_WITHOUT_TIMEZONE, + "domainMetadata" => WriterFeatures::DOMAIN_METADATA, + "v2Checkpoint" => WriterFeatures::V2_CHECKPOINT, + "icebergCompatV1" => WriterFeatures::ICEBERG_COMPAT_V1, + f => WriterFeatures::OTHER(f.to_string()), + }, + f => WriterFeatures::OTHER(f.to_string()), + } + } } /// The commitInfo is a fairly flexible action within the delta specification, where arbitrary data can be stored. @@ -830,6 +1013,23 @@ pub enum DeltaOperation { ///Datetime to restore datetime: Option, }, // TODO: Add more operations + + #[serde(rename_all = "camelCase")] + /// Represents the start of `Vacuum` operation + VacuumStart { + /// Whether the retention check is enforced + retention_check_enabled: bool, + /// The specified retetion period in milliseconds + specified_retention_millis: Option, + /// The default delta table retention milliseconds policy + default_retention_millis: i64, + }, + + /// Represents the end of `Vacuum` operation + VacuumEnd { + /// The status of the operation + status: String, + }, } impl DeltaOperation { @@ -837,9 +1037,10 @@ impl DeltaOperation { pub fn name(&self) -> &str { // operation names taken from https://learn.microsoft.com/en-us/azure/databricks/delta/history#--operation-metrics-keys match &self { - DeltaOperation::Create { mode, .. } if matches!(mode, SaveMode::Overwrite) => { - "CREATE OR REPLACE TABLE" - } + DeltaOperation::Create { + mode: SaveMode::Overwrite, + .. + } => "CREATE OR REPLACE TABLE", DeltaOperation::Create { .. } => "CREATE TABLE", DeltaOperation::Write { .. } => "WRITE", DeltaOperation::Delete { .. } => "DELETE", @@ -849,6 +1050,8 @@ impl DeltaOperation { DeltaOperation::Optimize { .. } => "OPTIMIZE", DeltaOperation::FileSystemCheck { .. } => "FSCK", DeltaOperation::Restore { .. } => "RESTORE", + DeltaOperation::VacuumStart { .. } => "VACUUM START", + DeltaOperation::VacuumEnd { .. } => "VACUUM END", } } @@ -884,7 +1087,7 @@ impl DeltaOperation { /// Denotes if the operation changes the data contained in the table pub fn changes_data(&self) -> bool { match self { - Self::Optimize { .. } => false, + Self::Optimize { .. } | Self::VacuumStart { .. } | Self::VacuumEnd { .. } => false, Self::Create { .. } | Self::FileSystemCheck {} | Self::StreamingUpdate { .. } diff --git a/rust/src/action/parquet2_read/boolean.rs b/crates/deltalake-core/src/protocol/parquet2_read/boolean.rs similarity index 98% rename from rust/src/action/parquet2_read/boolean.rs rename to crates/deltalake-core/src/protocol/parquet2_read/boolean.rs index 553ba36709..474a61a153 100644 --- a/rust/src/action/parquet2_read/boolean.rs +++ b/crates/deltalake-core/src/protocol/parquet2_read/boolean.rs @@ -4,7 +4,7 @@ use parquet2::page::DataPage; use super::validity::ValidityRowIndexIter; use super::{split_page, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; /// Parquet dictionary primitive value reader pub struct SomeBooleanValueIter<'a> { diff --git a/rust/src/action/parquet2_read/dictionary/binary.rs b/crates/deltalake-core/src/protocol/parquet2_read/dictionary/binary.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/binary.rs rename to crates/deltalake-core/src/protocol/parquet2_read/dictionary/binary.rs diff --git a/rust/src/action/parquet2_read/dictionary/mod.rs b/crates/deltalake-core/src/protocol/parquet2_read/dictionary/mod.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/mod.rs rename to crates/deltalake-core/src/protocol/parquet2_read/dictionary/mod.rs diff --git a/rust/src/action/parquet2_read/dictionary/primitive.rs b/crates/deltalake-core/src/protocol/parquet2_read/dictionary/primitive.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/primitive.rs rename to crates/deltalake-core/src/protocol/parquet2_read/dictionary/primitive.rs diff --git a/rust/src/action/parquet2_read/map.rs b/crates/deltalake-core/src/protocol/parquet2_read/map.rs similarity index 99% rename from rust/src/action/parquet2_read/map.rs rename to crates/deltalake-core/src/protocol/parquet2_read/map.rs index ed730e383b..0739feae2d 100644 --- a/rust/src/action/parquet2_read/map.rs +++ b/crates/deltalake-core/src/protocol/parquet2_read/map.rs @@ -3,7 +3,7 @@ use parquet2::page::{DataPage, DictPage}; use super::string::for_each_repeated_string_field_value_with_idx; use super::{ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; #[derive(Default)] pub struct MapState { diff --git a/rust/src/action/parquet2_read/mod.rs b/crates/deltalake-core/src/protocol/parquet2_read/mod.rs similarity index 96% rename from rust/src/action/parquet2_read/mod.rs rename to crates/deltalake-core/src/protocol/parquet2_read/mod.rs index afa6065279..ae5461d2b6 100644 --- a/rust/src/action/parquet2_read/mod.rs +++ b/crates/deltalake-core/src/protocol/parquet2_read/mod.rs @@ -10,8 +10,8 @@ use parquet2::read::decompress; use parquet2::read::get_page_iterator; use parquet2::read::levels::get_bit_width; -use super::ProtocolError; -use crate::action::{Action, Add, CommitInfo, MetaData, Protocol, Remove, Txn}; +use super::{ProtocolError, ReaderFeatures, WriterFeatures}; +use crate::protocol::{Action, Add, CommitInfo, MetaData, Protocol, Remove, Txn}; use crate::schema::Guid; use boolean::for_each_boolean_field_value; use map::for_each_map_field_value; @@ -26,7 +26,7 @@ mod stats; mod string; mod validity; -/// Parquet deserilization error +/// Parquet deserialization error #[derive(thiserror::Error, Debug)] pub enum ParseError { /// Generic parsing error @@ -614,6 +614,30 @@ fn deserialize_protocol_column_page( |action: &mut Protocol, v: i32| action.min_writer_version = v, )?; } + "readerFeatures" => { + for_each_repeated_string_field_value( + actions, + page, + dict, + descriptor, + |action: &mut Protocol, v: Vec| { + action.reader_features = + Some(v.into_iter().map(ReaderFeatures::from).collect()); + }, + )?; + } + "writerFeatures" => { + for_each_repeated_string_field_value( + actions, + page, + dict, + descriptor, + |action: &mut Protocol, v: Vec| { + action.writer_features = + Some(v.into_iter().map(WriterFeatures::from).collect()); + }, + )?; + } _ => { warn!("Unexpected field `{}` in protocol", f); } @@ -751,7 +775,7 @@ mod tests { assert_eq!(meta_data.description, None); assert_eq!( meta_data.format, - crate::action::Format::new("parquet".to_string(), None), + crate::protocol::Format::new("parquet".to_string(), None), ); assert_eq!(meta_data.schema_string, "{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"); assert_eq!(meta_data.partition_columns.len(), 0); @@ -826,7 +850,7 @@ mod tests { assert_eq!(meta_data.description, None); assert_eq!( meta_data.format, - crate::action::Format::new("parquet".to_string(), None), + crate::protocol::Format::new("parquet".to_string(), None), ); assert_eq!( meta_data.schema_string, diff --git a/rust/src/action/parquet2_read/primitive.rs b/crates/deltalake-core/src/protocol/parquet2_read/primitive.rs similarity index 99% rename from rust/src/action/parquet2_read/primitive.rs rename to crates/deltalake-core/src/protocol/parquet2_read/primitive.rs index 4f262f7aaa..29147ea8ca 100644 --- a/rust/src/action/parquet2_read/primitive.rs +++ b/crates/deltalake-core/src/protocol/parquet2_read/primitive.rs @@ -12,7 +12,7 @@ use parquet2::types::NativeType; use super::dictionary; use super::validity::ValidityRowIndexIter; use super::{split_page, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; struct ExactChunksIter<'a, T: NativeType> { chunks: std::slice::ChunksExact<'a, u8>, diff --git a/rust/src/action/parquet2_read/stats.rs b/crates/deltalake-core/src/protocol/parquet2_read/stats.rs similarity index 85% rename from rust/src/action/parquet2_read/stats.rs rename to crates/deltalake-core/src/protocol/parquet2_read/stats.rs index c9bb2f9bdc..689dfea6c0 100644 --- a/rust/src/action/parquet2_read/stats.rs +++ b/crates/deltalake-core/src/protocol/parquet2_read/stats.rs @@ -1,4 +1,4 @@ -use crate::action::{Add, ProtocolError, Stats}; +use crate::protocol::{Add, ProtocolError, Stats}; impl Add { /// Returns the composite HashMap representation of stats contained in the action if present. diff --git a/rust/src/action/parquet2_read/string.rs b/crates/deltalake-core/src/protocol/parquet2_read/string.rs similarity index 99% rename from rust/src/action/parquet2_read/string.rs rename to crates/deltalake-core/src/protocol/parquet2_read/string.rs index 1a851aec3b..fc0ec574e0 100644 --- a/rust/src/action/parquet2_read/string.rs +++ b/crates/deltalake-core/src/protocol/parquet2_read/string.rs @@ -9,7 +9,7 @@ use super::dictionary; use super::dictionary::binary::BinaryPageDict; use super::validity::{ValidityRepeatedRowIndexIter, ValidityRowIndexIter}; use super::{split_page, split_page_nested, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; pub trait StringValueIter<'a>: Iterator> { fn try_from_encoded_values( diff --git a/rust/src/action/parquet2_read/validity.rs b/crates/deltalake-core/src/protocol/parquet2_read/validity.rs similarity index 100% rename from rust/src/action/parquet2_read/validity.rs rename to crates/deltalake-core/src/protocol/parquet2_read/validity.rs diff --git a/rust/src/action/parquet_read/mod.rs b/crates/deltalake-core/src/protocol/parquet_read/mod.rs similarity index 98% rename from rust/src/action/parquet_read/mod.rs rename to crates/deltalake-core/src/protocol/parquet_read/mod.rs index 3ff6dfa710..d5e3e708b6 100644 --- a/rust/src/action/parquet_read/mod.rs +++ b/crates/deltalake-core/src/protocol/parquet_read/mod.rs @@ -6,7 +6,7 @@ use num_traits::cast::ToPrimitive; use parquet::record::{Field, ListAccessor, MapAccessor, RowAccessor}; use serde_json::json; -use crate::action::{ +use crate::protocol::{ Action, Add, AddCDCFile, ColumnCountStat, ColumnValueStat, DeletionVector, MetaData, Protocol, ProtocolError, Remove, Stats, Txn, }; @@ -610,6 +610,18 @@ impl Protocol { gen_action_type_error("protocol", "minWriterVersion", "int") })?; } + "readerFeatures" => { + re.reader_features = record + .get_list(i) + .map(|l| l.elements().iter().map(From::from).collect()) + .ok() + } + "writerFeatures" => { + re.writer_features = record + .get_list(i) + .map(|l| l.elements().iter().map(From::from).collect()) + .ok() + } _ => { log::debug!( "Unexpected field name `{}` for protocol action: {:?}", diff --git a/crates/deltalake-core/src/protocol/serde_path.rs b/crates/deltalake-core/src/protocol/serde_path.rs new file mode 100644 index 0000000000..9868523e81 --- /dev/null +++ b/crates/deltalake-core/src/protocol/serde_path.rs @@ -0,0 +1,89 @@ +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() +} + +fn decode_path(path: &str) -> Result { + Ok(percent_decode_str(path).decode_utf8()?.to_string()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_encode_path() { + let cases = [ + ( + "string=$%25&%2F()%3D%5E%22%5B%5D%23%2A%3F.%3A/part-00023-4b06bc90-0678-4a63-94a2-f09af1adb945.c000.snappy.parquet", + "string=$%2525&%252F()%253D%255E%2522%255B%255D%2523%252A%253F.%253A/part-00023-4b06bc90-0678-4a63-94a2-f09af1adb945.c000.snappy.parquet", + ), + ( + "string=$%25&%2F()%3D%5E%22<>~%5B%5D%7B}`%23|%2A%3F%2F%5Cr%5Cn.%3A/part-00023-e0a68495-8098-40a6-be5f-b502b111b789.c000.snappy.parquet", + "string=$%2525&%252F()%253D%255E%2522%3C%3E~%255B%255D%257B%7D%60%2523%7C%252A%253F%252F%255Cr%255Cn.%253A/part-00023-e0a68495-8098-40a6-be5f-b502b111b789.c000.snappy.parquet" + ), + ( + "string=$%25&%2F()%3D%5E%22<>~%5B%5D%7B}`%23|%2A%3F%2F%5Cr%5Cn.%3A_-/part-00023-346b6795-dafa-4948-bda5-ecdf4baa4445.c000.snappy.parquet", + "string=$%2525&%252F()%253D%255E%2522%3C%3E~%255B%255D%257B%7D%60%2523%7C%252A%253F%252F%255Cr%255Cn.%253A_-/part-00023-346b6795-dafa-4948-bda5-ecdf4baa4445.c000.snappy.parquet" + ) + ]; + + for (raw, expected) in cases { + let encoded = encode_path(raw); + assert_eq!(encoded, expected); + let decoded = decode_path(expected).unwrap(); + assert_eq!(decoded, raw); + } + } +} diff --git a/rust/src/time_utils.rs b/crates/deltalake-core/src/protocol/time_utils.rs similarity index 100% rename from rust/src/time_utils.rs rename to crates/deltalake-core/src/protocol/time_utils.rs diff --git a/rust/src/delta_arrow.rs b/crates/deltalake-core/src/schema/arrow_convert.rs similarity index 99% rename from rust/src/delta_arrow.rs rename to crates/deltalake-core/src/schema/arrow_convert.rs index 99d7d67732..2b37b05c4a 100644 --- a/rust/src/delta_arrow.rs +++ b/crates/deltalake-core/src/schema/arrow_convert.rs @@ -601,8 +601,8 @@ fn max_min_schema_for_fields(dest: &mut Vec, f: &ArrowField) { true, )); } - // don't compute min or max for list or map types - ArrowDataType::List(_) | ArrowDataType::Map(_, _) => { /* noop */ } + // don't compute min or max for list, map or binary types + ArrowDataType::List(_) | ArrowDataType::Map(_, _) | ArrowDataType::Binary => { /* noop */ } _ => { let f = f.clone(); dest.push(f); @@ -989,6 +989,7 @@ mod tests { ), true, ), + ArrowField::new("binary", ArrowDataType::Binary, true), ]; let expected = vec![fields[0].clone(), fields[1].clone()]; diff --git a/rust/src/schema.rs b/crates/deltalake-core/src/schema/mod.rs similarity index 99% rename from rust/src/schema.rs rename to crates/deltalake-core/src/schema/mod.rs index 2602c5cd68..a853725fc6 100644 --- a/rust/src/schema.rs +++ b/crates/deltalake-core/src/schema/mod.rs @@ -8,6 +8,10 @@ use std::collections::HashMap; use crate::errors::DeltaTableError; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod arrow_convert; +pub mod partitions; + /// Type alias for a string expected to match a GUID/UUID format pub type Guid = String; diff --git a/rust/src/partitions.rs b/crates/deltalake-core/src/schema/partitions.rs similarity index 65% rename from rust/src/partitions.rs rename to crates/deltalake-core/src/schema/partitions.rs index ed4a5a2eaf..3750038b3a 100644 --- a/rust/src/partitions.rs +++ b/crates/deltalake-core/src/schema/partitions.rs @@ -2,39 +2,39 @@ use std::convert::TryFrom; -use super::schema::SchemaDataType; +use super::SchemaDataType; use crate::errors::DeltaTableError; use std::cmp::Ordering; use std::collections::HashMap; /// A Enum used for selecting the partition value operation when filtering a DeltaTable partition. #[derive(Clone, Debug, PartialEq, Eq)] -pub enum PartitionValue { +pub enum PartitionValue { /// The partition value with the equal operator - Equal(T), + Equal(String), /// The partition value with the not equal operator - NotEqual(T), + NotEqual(String), /// The partition value with the greater than operator - GreaterThan(T), + GreaterThan(String), /// The partition value with the greater than or equal operator - GreaterThanOrEqual(T), + GreaterThanOrEqual(String), /// The partition value with the less than operator - LessThan(T), + LessThan(String), /// The partition value with the less than or equal operator - LessThanOrEqual(T), + LessThanOrEqual(String), /// The partition values with the in operator - In(Vec), + In(Vec), /// The partition values with the not in operator - NotIn(Vec), + NotIn(Vec), } /// A Struct used for filtering a DeltaTable partition by key and value. #[derive(Clone, Debug, PartialEq, Eq)] -pub struct PartitionFilter<'a, T> { +pub struct PartitionFilter { /// The key of the PartitionFilter - pub key: &'a str, + pub key: String, /// The value of the PartitionFilter - pub value: PartitionValue, + pub value: PartitionValue, } fn compare_typed_value( @@ -65,11 +65,11 @@ fn compare_typed_value( } /// Partition filters methods for filtering the DeltaTable partitions. -impl<'a> PartitionFilter<'a, &str> { +impl PartitionFilter { /// Indicates if a DeltaTable partition matches with the partition filter by key and value. pub fn match_partition( &self, - partition: &DeltaTablePartition<'a>, + partition: &DeltaTablePartition, data_type: &SchemaDataType, ) -> bool { if self.key != partition.key { @@ -80,22 +80,22 @@ impl<'a> PartitionFilter<'a, &str> { PartitionValue::Equal(value) => value == &partition.value, PartitionValue::NotEqual(value) => value != &partition.value, PartitionValue::GreaterThan(value) => { - compare_typed_value(partition.value, value.to_owned(), data_type) + compare_typed_value(&partition.value, value, data_type) .map(|x| x.is_gt()) .unwrap_or(false) } PartitionValue::GreaterThanOrEqual(value) => { - compare_typed_value(partition.value, value.to_owned(), data_type) + compare_typed_value(&partition.value, value, data_type) .map(|x| x.is_ge()) .unwrap_or(false) } PartitionValue::LessThan(value) => { - compare_typed_value(partition.value, value.to_owned(), data_type) + compare_typed_value(&partition.value, value, data_type) .map(|x| x.is_lt()) .unwrap_or(false) } PartitionValue::LessThanOrEqual(value) => { - compare_typed_value(partition.value, value.to_owned(), data_type) + compare_typed_value(&partition.value, value, data_type) .map(|x| x.is_le()) .unwrap_or(false) } @@ -108,10 +108,13 @@ impl<'a> PartitionFilter<'a, &str> { /// matches with the partition filter. pub fn match_partitions( &self, - partitions: &[DeltaTablePartition<'a>], + partitions: &[DeltaTablePartition], partition_col_data_types: &HashMap<&str, &SchemaDataType>, ) -> bool { - let data_type = partition_col_data_types.get(self.key).unwrap().to_owned(); + let data_type = partition_col_data_types + .get(self.key.as_str()) + .unwrap() + .to_owned(); partitions .iter() .any(|partition| self.match_partition(partition, data_type)) @@ -119,36 +122,36 @@ impl<'a> PartitionFilter<'a, &str> { } /// Create a PartitionFilter from a filter Tuple with the structure (key, operation, value). -impl<'a, T: std::fmt::Debug> TryFrom<(&'a str, &str, T)> for PartitionFilter<'a, T> { +impl TryFrom<(&str, &str, &str)> for PartitionFilter { type Error = DeltaTableError; /// Try to create a PartitionFilter from a Tuple of (key, operation, value). /// Returns a DeltaTableError in case of a malformed filter. - fn try_from(filter: (&'a str, &str, T)) -> Result { + fn try_from(filter: (&str, &str, &str)) -> Result { match filter { (key, "=", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::Equal(value), + key: key.to_owned(), + value: PartitionValue::Equal(value.to_owned()), }), (key, "!=", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::NotEqual(value), + key: key.to_owned(), + value: PartitionValue::NotEqual(value.to_owned()), }), (key, ">", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::GreaterThan(value), + key: key.to_owned(), + value: PartitionValue::GreaterThan(value.to_owned()), }), (key, ">=", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::GreaterThanOrEqual(value), + key: key.to_owned(), + value: PartitionValue::GreaterThanOrEqual(value.to_owned()), }), (key, "<", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::LessThan(value), + key: key.to_owned(), + value: PartitionValue::LessThan(value.to_owned()), }), (key, "<=", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::LessThanOrEqual(value), + key: key.to_owned(), + value: PartitionValue::LessThanOrEqual(value.to_owned()), }), (_, _, _) => Err(DeltaTableError::InvalidPartitionFilter { partition_filter: format!("{filter:?}"), @@ -158,20 +161,20 @@ impl<'a, T: std::fmt::Debug> TryFrom<(&'a str, &str, T)> for PartitionFilter<'a, } /// Create a PartitionFilter from a filter Tuple with the structure (key, operation, list(value)). -impl<'a, T: std::fmt::Debug> TryFrom<(&'a str, &str, Vec)> for PartitionFilter<'a, T> { +impl TryFrom<(&str, &str, &[&str])> for PartitionFilter { type Error = DeltaTableError; /// Try to create a PartitionFilter from a Tuple of (key, operation, list(value)). /// Returns a DeltaTableError in case of a malformed filter. - fn try_from(filter: (&'a str, &str, Vec)) -> Result { + fn try_from(filter: (&str, &str, &[&str])) -> Result { match filter { (key, "in", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::In(value), + key: key.to_owned(), + value: PartitionValue::In(value.iter().map(|x| x.to_string()).collect()), }), (key, "not in", value) if !key.is_empty() => Ok(PartitionFilter { - key, - value: PartitionValue::NotIn(value), + key: key.to_owned(), + value: PartitionValue::NotIn(value.iter().map(|x| x.to_string()).collect()), }), (_, _, _) => Err(DeltaTableError::InvalidPartitionFilter { partition_filter: format!("{filter:?}"), @@ -182,38 +185,36 @@ impl<'a, T: std::fmt::Debug> TryFrom<(&'a str, &str, Vec)> for PartitionFilte /// A Struct DeltaTablePartition used to represent a partition of a DeltaTable. #[derive(Clone, Debug, PartialEq, Eq)] -pub struct DeltaTablePartition<'a> { +pub struct DeltaTablePartition { /// The key of the DeltaTable partition. - pub key: &'a str, + pub key: String, /// The value of the DeltaTable partition. - pub value: &'a str, + pub value: String, } -/** - * Create a DeltaTable partition from a HivePartition string. - * - * A HivePartition string is represented by a "key=value" format. - * - * ```rust - * use deltalake::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<'a> TryFrom<&'a str> for DeltaTablePartition<'a> { +/// 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: &'a str) -> Result { + 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], - value: partition_splitted[1], + key: partition_splitted[0].to_owned(), + value: partition_splitted[1].to_owned(), }), _ => Err(DeltaTableError::PartitionError { partition: partition.to_string(), @@ -222,31 +223,32 @@ impl<'a> TryFrom<&'a str> for DeltaTablePartition<'a> { } } -impl<'a> DeltaTablePartition<'a> { - /** - * Try to create a DeltaTable partition from a partition value kv pair. - * - * ```rust - * use deltalake::DeltaTablePartition; - * - * let value = (&"ds".to_string(), &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); - * ``` - */ +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: (&'a String, &'a Option), - default_for_null: &'a str, + partition_value: (&str, &Option), + default_for_null: &str, ) -> Self { let (k, v) = partition_value; let v = match v { Some(s) => s, None => default_for_null, }; - DeltaTablePartition { key: k, value: v } + DeltaTablePartition { + key: k.to_owned(), + value: v.to_owned(), + } } } diff --git a/crates/deltalake-core/src/storage/config.rs b/crates/deltalake-core/src/storage/config.rs new file mode 100644 index 0000000000..1cba57b579 --- /dev/null +++ b/crates/deltalake-core/src/storage/config.rs @@ -0,0 +1,332 @@ +//! Configuration handling for defining Storage backends for DeltaTables. +use std::collections::HashMap; +use std::sync::Arc; + +use object_store::memory::InMemory; +use object_store::path::Path; +use object_store::prefix::PrefixStore; +use object_store::{parse_url_opts, DynObjectStore, Error as ObjectStoreError, ObjectStore}; +use serde::{Deserialize, Serialize}; +use url::Url; + +use super::file::FileStorageBackend; +use super::utils::str_is_truthy; +use crate::errors::{DeltaResult, DeltaTableError}; + +#[cfg(any(feature = "s3", feature = "s3-native-tls"))] +use super::s3::{S3StorageBackend, S3StorageOptions}; +#[cfg(feature = "hdfs")] +use datafusion_objectstore_hdfs::object_store::hdfs::HadoopFileSystem; +#[cfg(any(feature = "s3", feature = "s3-native-tls"))] +use object_store::aws::AmazonS3ConfigKey; +#[cfg(feature = "azure")] +use object_store::azure::AzureConfigKey; +#[cfg(feature = "gcs")] +use object_store::gcp::GoogleConfigKey; +#[cfg(any( + feature = "s3", + feature = "s3-native-tls", + feature = "gcs", + feature = "azure" +))] +use std::str::FromStr; + +#[cfg(feature = "azure")] +mod azure; + +/// Recognises various URL formats, identifying the relevant [`ObjectStore`](crate::ObjectStore) +#[derive(Debug, Eq, PartialEq)] +enum ObjectStoreScheme { + /// Url corresponding to LocalFileSystem + Local, + /// Url corresponding to InMemory + Memory, + /// Url corresponding to S3 + AmazonS3, + /// Url corresponding to GoogleCloudStorage + GoogleCloudStorage, + /// Url corresponding to MicrosoftAzure + MicrosoftAzure, + /// Url corresponding to HttpStore + Http, + /// Url corresponding to Hdfs + Hdfs, +} + +impl ObjectStoreScheme { + /// Create an [`ObjectStoreScheme`] from the provided [`Url`] + /// + /// Returns the [`ObjectStoreScheme`] and the remaining [`Path`] + fn parse( + url: &Url, + #[allow(unused)] options: &mut StorageOptions, + ) -> Result<(Self, Path), ObjectStoreError> { + let strip_bucket = || Some(url.path().strip_prefix('/')?.split_once('/')?.1); + + let (scheme, path) = match (url.scheme(), url.host_str()) { + ("file", None) => (Self::Local, url.path()), + ("memory", None) => (Self::Memory, url.path()), + ("s3" | "s3a", Some(_)) => (Self::AmazonS3, url.path()), + ("gs", Some(_)) => (Self::GoogleCloudStorage, url.path()), + ("az" | "adl" | "azure" | "abfs" | "abfss", Some(_)) => { + (Self::MicrosoftAzure, url.path()) + } + ("http", Some(_)) => (Self::Http, url.path()), + ("hdfs", Some(_)) => (Self::Hdfs, url.path()), + ("https", Some(host)) => { + if host.ends_with("dfs.core.windows.net") || host.ends_with("blob.core.windows.net") + { + (Self::MicrosoftAzure, url.path()) + } else if host.contains("dfs.fabric.microsoft.com") + || host.contains("blob.fabric.microsoft.com") + { + #[cfg(feature = "azure")] + if !options + .as_azure_options() + .contains_key(&AzureConfigKey::UseFabricEndpoint) + { + options.0.insert( + AzureConfigKey::UseFabricEndpoint.as_ref().to_string(), + "true".to_string(), + ); + } + (Self::MicrosoftAzure, url.path()) + } else if host.ends_with("amazonaws.com") { + match host.starts_with("s3") { + true => (Self::AmazonS3, strip_bucket().unwrap_or_default()), + false => (Self::AmazonS3, url.path()), + } + } else if host.ends_with("r2.cloudflarestorage.com") { + (Self::AmazonS3, strip_bucket().unwrap_or_default()) + } else { + (Self::Http, url.path()) + } + } + _ => return Err(ObjectStoreError::NotImplemented), + }; + + let path = Path::parse(path)?; + Ok((scheme, path)) + } +} + +/// Options used for configuring backend storage +#[derive(Clone, Debug, Serialize, Deserialize, Default)] +pub struct StorageOptions(pub HashMap); + +impl StorageOptions { + /// Create a new instance of [`StorageOptions`] + pub fn new(options: HashMap) -> Self { + let mut options = options; + if let Ok(value) = std::env::var("AZURE_STORAGE_ALLOW_HTTP") { + options.insert("allow_http".into(), value); + } + if let Ok(value) = std::env::var("AZURE_STORAGE_USE_HTTP") { + options.insert("allow_http".into(), value); + } + if let Ok(value) = std::env::var("AWS_ALLOW_HTTP") { + options.insert("allow_http".into(), value); + } + Self(options) + } + + /// Add values from the environment to storage options + #[cfg(feature = "azure")] + pub fn with_env_azure(&mut self) { + 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) = AzureConfigKey::from_str(&key.to_ascii_lowercase()) { + if !self.0.contains_key(config_key.as_ref()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + } + + /// Add values from the environment to storage options + #[cfg(feature = "gcs")] + pub fn with_env_gcs(&mut self) { + 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) = GoogleConfigKey::from_str(&key.to_ascii_lowercase()) { + if !self.0.contains_key(config_key.as_ref()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + } + + /// Add values from the environment to storage options + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + pub fn with_env_s3(&mut self) { + 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 !self.0.contains_key(config_key.as_ref()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + } + + /// Denotes if unsecure connections via http are allowed + pub fn allow_http(&self) -> bool { + self.0.iter().any(|(key, value)| { + key.to_ascii_lowercase().contains("allow_http") & str_is_truthy(value) + }) + } + + /// Subset of options relevant for azure storage + #[cfg(feature = "azure")] + pub fn as_azure_options(&self) -> HashMap { + self.0 + .iter() + .filter_map(|(key, value)| { + let az_key = AzureConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; + Some((az_key, value.clone())) + }) + .collect() + } + + /// Subset of options relevant for s3 storage + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + pub fn as_s3_options(&self) -> HashMap { + self.0 + .iter() + .filter_map(|(key, value)| { + let s3_key = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; + Some((s3_key, value.clone())) + }) + .collect() + } + + /// Subset of options relevant for gcs storage + #[cfg(feature = "gcs")] + pub fn as_gcs_options(&self) -> HashMap { + self.0 + .iter() + .filter_map(|(key, value)| { + let gcs_key = GoogleConfigKey::from_str(&key.to_ascii_lowercase()).ok()?; + Some((gcs_key, value.clone())) + }) + .collect() + } +} + +impl From> for StorageOptions { + fn from(value: HashMap) -> Self { + Self::new(value) + } +} + +pub(crate) fn configure_store( + url: &Url, + options: &mut StorageOptions, +) -> DeltaResult> { + let (scheme, _prefix) = ObjectStoreScheme::parse(url, options)?; + match scheme { + ObjectStoreScheme::Local => { + let path = url + .to_file_path() + .map_err(|_| DeltaTableError::InvalidTableLocation(url.to_string()))?; + Ok(Arc::new(FileStorageBackend::try_new(path)?)) + } + ObjectStoreScheme::Memory => url_prefix_handler(InMemory::new(), Path::parse(url.path())?), + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + ObjectStoreScheme::AmazonS3 => { + options.with_env_s3(); + let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; + if options + .as_s3_options() + .contains_key(&AmazonS3ConfigKey::CopyIfNotExists) + { + url_prefix_handler(store, prefix) + } else { + let store = S3StorageBackend::try_new( + Arc::new(store), + S3StorageOptions::from_map(&options.0), + )?; + url_prefix_handler(store, prefix) + } + } + #[cfg(feature = "azure")] + ObjectStoreScheme::MicrosoftAzure => { + let config = azure::AzureConfigHelper::try_new(options.as_azure_options())?.build()?; + let (store, prefix) = parse_url_opts(url, config)?; + url_prefix_handler(store, prefix) + } + #[cfg(feature = "gcs")] + ObjectStoreScheme::GoogleCloudStorage => { + options.with_env_gcs(); + let (store, prefix) = parse_url_opts(url, options.as_gcs_options())?; + url_prefix_handler(store, prefix) + } + #[cfg(feature = "hdfs")] + ObjectStoreScheme::Hdfs => { + let store = HadoopFileSystem::new(url.as_ref()).ok_or_else(|| { + DeltaTableError::Generic(format!( + "failed to create HadoopFileSystem for {}", + url.as_ref() + )) + })?; + url_prefix_handler(store, _prefix) + } + #[cfg(not(feature = "hdfs"))] + ObjectStoreScheme::Hdfs => Err(DeltaTableError::MissingFeature { + feature: "hdfs", + url: url.as_ref().into(), + }), + _ => { + let (store, prefix) = parse_url_opts(url, options.0.clone())?; + url_prefix_handler(store, prefix) + } + } +} + +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 test { + use crate::table::builder::ensure_table_uri; + + use super::*; + + #[tokio::test] + async fn test_configure_store_local() -> Result<(), Box> { + let temp_dir = tempfile::tempdir().unwrap(); + let temp_dir_path = temp_dir.path(); + let path = temp_dir_path.join("test space 😁"); + + let table_uri = ensure_table_uri(path.as_os_str().to_str().unwrap()).unwrap(); + + let store = configure_store(&table_uri, &mut StorageOptions::default()).unwrap(); + + let contents = b"test"; + let key = "test.txt"; + let file_path = path.join(key); + std::fs::write(&file_path, contents).unwrap(); + + let res = store + .get(&object_store::path::Path::from(key)) + .await + .unwrap() + .bytes() + .await + .unwrap(); + assert_eq!(res.as_ref(), contents); + + Ok(()) + } +} diff --git a/crates/deltalake-core/src/storage/config/azure.rs b/crates/deltalake-core/src/storage/config/azure.rs new file mode 100644 index 0000000000..e556b9eb6e --- /dev/null +++ b/crates/deltalake-core/src/storage/config/azure.rs @@ -0,0 +1,226 @@ +//! Auxiliary module for generating a valig Azure configuration. +//! +//! Azure offers many different 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::azure::AzureConfigKey; +use object_store::Error; + +use crate::{DeltaResult, DeltaTableError}; + +lazy_static::lazy_static! { + static ref CREDENTIAL_KEYS: Vec = + Vec::from_iter([ + AzureConfigKey::ClientId, + AzureConfigKey::ClientSecret, + AzureConfigKey::FederatedTokenFile, + AzureConfigKey::SasKey, + AzureConfigKey::Token, + AzureConfigKey::MsiEndpoint, + AzureConfigKey::ObjectId, + AzureConfigKey::MsiResourceId, + ]); +} + +/// Credential +enum AzureCredential { + /// Using the account master key + AccessKey, + /// Using a static bearer token + BearerToken, + /// Authorizing with secret + ClientSecret, + /// Using a shared access signature + ManagedIdentity, + /// Using a shared access signature + SasKey, + /// Using workload identity + 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 + fn keys(&self) -> Vec { + match self { + Self::AccessKey => Vec::from_iter([AzureConfigKey::AccessKey]), + Self::BearerToken => Vec::from_iter([AzureConfigKey::Token]), + Self::ClientSecret => Vec::from_iter([ + AzureConfigKey::ClientId, + AzureConfigKey::ClientSecret, + AzureConfigKey::AuthorityId, + ]), + Self::WorkloadIdentity => Vec::from_iter([ + AzureConfigKey::AuthorityId, + AzureConfigKey::ClientId, + AzureConfigKey::FederatedTokenFile, + ]), + Self::SasKey => Vec::from_iter([AzureConfigKey::SasKey]), + Self::ManagedIdentity => Vec::new(), + } + } +} + +/// 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 AzureConfigHelper { + config: HashMap, + env_config: HashMap, + priority: Vec, +} + +impl AzureConfigHelper { + /// Create a new [`ConfigHelper`] + pub fn try_new( + config: impl IntoIterator, impl Into)>, + ) -> DeltaResult { + 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("AZURE_") { + if let Ok(config_key) = AzureConfigKey::from_str(&key.to_ascii_lowercase()) { + env_config.insert(config_key, value.to_string()); + } + } + } + } + + Ok(Self { + config: config + .into_iter() + .map(|(key, value)| Ok((AzureConfigKey::from_str(key.as_ref())?, value.into()))) + .collect::>()?, + env_config, + priority: Vec::from_iter([ + AzureCredential::AccessKey, + AzureCredential::SasKey, + AzureCredential::BearerToken, + AzureCredential::ClientSecret, + AzureCredential::WorkloadIdentity, + ]), + }) + } + + /// Check if all credential keys are contained in passed config + fn has_full_config(&self, cred: &AzureCredential) -> 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: &AzureCredential) -> 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: &AzureCredential) -> 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) -> DeltaResult> { + let mut has_credential = false; + + if self.config.contains_key(&AzureConfigKey::UseAzureCli) { + has_credential = true; + } + + // 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/rust/src/storage/file.rs b/crates/deltalake-core/src/storage/file.rs similarity index 100% rename from rust/src/storage/file.rs rename to crates/deltalake-core/src/storage/file.rs diff --git a/rust/src/storage/mod.rs b/crates/deltalake-core/src/storage/mod.rs similarity index 98% rename from rust/src/storage/mod.rs rename to crates/deltalake-core/src/storage/mod.rs index 9798a10b1a..c7309531ea 100644 --- a/rust/src/storage/mod.rs +++ b/crates/deltalake-core/src/storage/mod.rs @@ -89,10 +89,10 @@ impl DeltaObjectStore { /// # Arguments /// /// * `location` - A url pointing to the root of the delta table. - /// * `options` - Options passed to underlying builders. See [`with_storage_options`](crate::builder::DeltaTableBuilder::with_storage_options) + /// * `options` - Options passed to underlying builders. See [`with_storage_options`](crate::table::builder::DeltaTableBuilder::with_storage_options) pub fn try_new(location: Url, options: impl Into + Clone) -> DeltaResult { - let options = options.into(); - let storage = config::configure_store(&location, &options)?; + let mut options = options.into(); + let storage = config::configure_store(&location, &mut options)?; Ok(Self { storage, location, diff --git a/rust/src/storage/s3.rs b/crates/deltalake-core/src/storage/s3.rs similarity index 98% rename from rust/src/storage/s3.rs rename to crates/deltalake-core/src/storage/s3.rs index b7da2ae4d7..23e091dea5 100644 --- a/rust/src/storage/s3.rs +++ b/crates/deltalake-core/src/storage/s3.rs @@ -1,7 +1,7 @@ //! AWS S3 storage backend. use super::utils::str_is_truthy; -use crate::builder::{s3_storage_options, str_option}; +use crate::table::builder::{s3_storage_options, str_option}; use bytes::Bytes; use dynamodb_lock::{DynamoError, LockClient, LockItem, DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS}; use futures::stream::BoxStream; @@ -383,7 +383,7 @@ impl S3StorageBackend { /// /// ```rust /// use object_store::aws::AmazonS3Builder; - /// use deltalake::storage::s3::{S3StorageBackend, S3StorageOptions}; + /// use deltalake_core::storage::s3::{S3StorageBackend, S3StorageOptions}; /// use std::sync::Arc; /// /// let inner = AmazonS3Builder::new() @@ -474,6 +474,14 @@ impl ObjectStore for S3StorageBackend { self.inner.list(prefix).await } + async fn list_with_offset( + &self, + prefix: Option<&Path>, + offset: &Path, + ) -> ObjectStoreResult>> { + self.inner.list_with_offset(prefix, offset).await + } + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { self.inner.list_with_delimiter(prefix).await } diff --git a/rust/src/storage/utils.rs b/crates/deltalake-core/src/storage/utils.rs similarity index 95% rename from rust/src/storage/utils.rs rename to crates/deltalake-core/src/storage/utils.rs index 7cb27d721a..80710efd9b 100644 --- a/rust/src/storage/utils.rs +++ b/crates/deltalake-core/src/storage/utils.rs @@ -8,9 +8,9 @@ use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; -use crate::action::Add; -use crate::builder::DeltaTableBuilder; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; +use crate::table::builder::DeltaTableBuilder; /// Copies the contents from the `from` location into the `to` location pub async fn copy_table( @@ -82,7 +82,7 @@ impl TryFrom<&Add> for ObjectMeta { fn try_from(value: &Add) -> DeltaResult { let last_modified = Utc.from_utc_datetime( &NaiveDateTime::from_timestamp_millis(value.modification_time).ok_or( - DeltaTableError::from(crate::action::ProtocolError::InvalidField(format!( + DeltaTableError::from(crate::protocol::ProtocolError::InvalidField(format!( "invalid modification_time: {:?}", value.modification_time ))), diff --git a/rust/src/builder.rs b/crates/deltalake-core/src/table/builder.rs similarity index 99% rename from rust/src/builder.rs rename to crates/deltalake-core/src/table/builder.rs index ca7661800c..92fc4851ad 100644 --- a/rust/src/builder.rs +++ b/crates/deltalake-core/src/table/builder.rs @@ -9,7 +9,7 @@ use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; use url::Url; -use crate::delta::DeltaTable; +use super::DeltaTable; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::config::StorageOptions; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; @@ -341,7 +341,7 @@ pub mod s3_storage_options { /// See also . pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; /// Allow http connections - mainly useful for integration tests - pub const AWS_STORAGE_ALLOW_HTTP: &str = "AWS_STORAGE_ALLOW_HTTP"; + 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. diff --git a/rust/src/delta_config.rs b/crates/deltalake-core/src/table/config.rs similarity index 99% rename from rust/src/delta_config.rs rename to crates/deltalake-core/src/table/config.rs index 1b7f6b7e0f..60498767ab 100644 --- a/rust/src/delta_config.rs +++ b/crates/deltalake-core/src/table/config.rs @@ -387,7 +387,7 @@ fn parse_int(value: &str) -> Result { #[cfg(test)] mod tests { use super::*; - use crate::DeltaTableMetaData; + use crate::table::DeltaTableMetaData; use crate::Schema; use std::collections::HashMap; diff --git a/rust/src/delta.rs b/crates/deltalake-core/src/table/mod.rs similarity index 91% rename from rust/src/delta.rs rename to crates/deltalake-core/src/table/mod.rs index d877b77191..0a1e3116f1 100644 --- a/rust/src/delta.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -1,7 +1,5 @@ //! Delta Table read and write implementation -// Reference: https://github.com/delta-io/delta/blob/master/PROTOCOL.md -// use std::collections::HashMap; use std::convert::TryFrom; use std::fmt; @@ -21,17 +19,23 @@ use serde::ser::SerializeSeq; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use uuid::Uuid; -use super::action; -use super::action::{find_latest_check_point_for_version, get_last_checkpoint, Action}; -use super::partitions::PartitionFilter; -use super::schema::*; -use super::table_state::DeltaTableState; -use crate::action::{Add, ProtocolError, Stats}; +use self::builder::DeltaTableConfig; +use self::state::DeltaTableState; use crate::errors::DeltaTableError; +use crate::partitions::PartitionFilter; +use crate::protocol::{ + self, find_latest_check_point_for_version, get_last_checkpoint, Action, ReaderFeatures, + WriterFeatures, +}; +use crate::protocol::{Add, ProtocolError, Stats}; +use crate::schema::*; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; -// TODO re-exports only for transition -pub use crate::builder::{DeltaTableBuilder, DeltaTableConfig, DeltaVersion}; +pub mod builder; +pub mod config; +pub mod state; +#[cfg(feature = "arrow")] +pub mod state_arrow; /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] @@ -136,7 +140,7 @@ pub struct DeltaTableMetaData { /// User-provided description for this table pub description: Option, /// Specification of the encoding for the files stored in the table - pub format: action::Format, + pub format: protocol::Format, /// Schema of the table pub schema: Schema, /// An array containing the names of columns by which the data should be partitioned @@ -152,7 +156,7 @@ impl DeltaTableMetaData { pub fn new( name: Option, description: Option, - format: Option, + format: Option, schema: Schema, partition_columns: Vec, configuration: HashMap>, @@ -208,10 +212,10 @@ impl fmt::Display for DeltaTableMetaData { } } -impl TryFrom for DeltaTableMetaData { +impl TryFrom for DeltaTableMetaData { type Error = ProtocolError; - fn try_from(action_metadata: action::MetaData) -> Result { + fn try_from(action_metadata: protocol::MetaData) -> Result { let schema = action_metadata.get_schema()?; Ok(Self { id: action_metadata.id, @@ -414,8 +418,8 @@ impl DeltaTable { Ok(()) } - - async fn get_latest_version(&mut self) -> Result { + /// returns the latest available version of the table + pub async fn get_latest_version(&mut self) -> Result { let version_start = match get_last_checkpoint(&self.storage).await { Ok(last_check_point) => last_check_point.version, Err(ProtocolError::CheckpointNotFound) => { @@ -663,7 +667,7 @@ impl DeltaTable { pub async fn history( &mut self, limit: Option, - ) -> Result, DeltaTableError> { + ) -> 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?, @@ -709,8 +713,8 @@ impl DeltaTable { /// Obtain Add actions for files that match the filter pub fn get_active_add_actions_by_partitions<'a>( &'a self, - filters: &'a [PartitionFilter<'a, &'a str>], - ) -> Result + '_, DeltaTableError> { + filters: &'a [PartitionFilter], + ) -> Result + '_, DeltaTableError> { self.state.get_active_add_actions_by_partitions(filters) } @@ -718,7 +722,7 @@ impl DeltaTable { /// `PartitionFilter`s. pub fn get_files_by_partitions( &self, - filters: &[PartitionFilter<&str>], + filters: &[PartitionFilter], ) -> Result, DeltaTableError> { Ok(self .get_active_add_actions_by_partitions(filters)? @@ -735,7 +739,7 @@ impl DeltaTable { /// Return the file uris as strings for the partition(s) pub fn get_file_uris_by_partitions( &self, - filters: &[PartitionFilter<&str>], + filters: &[PartitionFilter], ) -> Result, DeltaTableError> { let files = self.get_files_by_partitions(filters)?; Ok(files @@ -796,7 +800,7 @@ impl DeltaTable { } /// Returns a vector of active tombstones (i.e. `Remove` actions present in the current delta log). - pub fn get_tombstones(&self) -> impl Iterator { + pub fn get_tombstones(&self) -> impl Iterator { self.state.unexpired_tombstones() } @@ -817,6 +821,16 @@ impl DeltaTable { self.state.min_writer_version() } + /// Returns current supported reader features by this table + pub fn get_reader_features(&self) -> Option<&HashSet> { + self.state.reader_features() + } + + /// Returns current supported writer features by this table + pub fn get_writer_features(&self) -> Option<&HashSet> { + self.state.writer_features() + } + /// 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<&Schema> { @@ -907,64 +921,12 @@ impl std::fmt::Debug for DeltaTable { } } -/// 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. -pub async fn open_table(table_uri: impl AsRef) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri).load().await?; - Ok(table) -} - -/// Same as `open_table`, but also accepts storage options to aid in building the table for a deduced -/// `StorageService`. -pub async fn open_table_with_storage_options( - table_uri: impl AsRef, - storage_options: HashMap, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_storage_options(storage_options) - .load() - .await?; - Ok(table) -} - -/// Creates a DeltaTable from the given path and loads it with the metadata from the given version. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table_with_version( - table_uri: impl AsRef, - version: i64, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_version(version) - .load() - .await?; - Ok(table) -} - -/// Creates a DeltaTable from the given path. -/// Loads metadata from the version appropriate based on the given ISO-8601/RFC-3339 timestamp. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table_with_ds( - table_uri: impl AsRef, - ds: impl AsRef, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_datestring(ds)? - .load() - .await?; - Ok(table) -} - -/// Returns rust crate version, can be use used in language bindings to expose Rust core version -pub fn crate_version() -> &'static str { - env!("CARGO_PKG_VERSION") -} - #[cfg(test)] mod tests { use super::*; - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - use crate::builder::DeltaTableBuilder; use crate::operations::create::CreateBuilder; + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + use crate::table::builder::DeltaTableBuilder; use pretty_assertions::assert_eq; use std::collections::HashMap; use tempdir::TempDir; diff --git a/rust/src/table_state.rs b/crates/deltalake-core/src/table/state.rs similarity index 85% rename from rust/src/table_state.rs rename to crates/deltalake-core/src/table/state.rs index 9be2200d9e..af4ff6369d 100644 --- a/rust/src/table_state.rs +++ b/crates/deltalake-core/src/table/state.rs @@ -10,14 +10,15 @@ use lazy_static::lazy_static; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use serde::{Deserialize, Serialize}; -use crate::action::{self, Action, Add, ProtocolError}; -use crate::delta_config::TableConfig; +use super::config::TableConfig; use crate::errors::DeltaTableError; use crate::partitions::{DeltaTablePartition, PartitionFilter}; +use crate::protocol::{self, Action, Add, ProtocolError, ReaderFeatures, WriterFeatures}; use crate::schema::SchemaDataType; use crate::storage::commit_uri_from_version; +use crate::table::DeltaTableMetaData; +use crate::DeltaTable; use crate::Schema; -use crate::{DeltaTable, DeltaTableMetaData}; #[cfg(any(feature = "parquet", feature = "parquet2"))] use super::{CheckPoint, DeltaTableConfig}; @@ -30,16 +31,18 @@ pub struct DeltaTableState { 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, + tombstones: HashSet, // active files for table state - files: Vec, + files: Vec, // Information added to individual commits - commit_infos: Vec, + commit_infos: Vec, // Domain metadatas provided by the system or user - domain_metadatas: Vec, + domain_metadatas: Vec, app_transaction_version: HashMap, min_reader_version: i32, min_writer_version: i32, + reader_features: Option>, + writer_features: Option>, // table metadata corresponding to current version current_metadata: Option, // retention period for tombstones in milli-seconds @@ -75,7 +78,7 @@ impl DeltaTableState { let mut new_state = DeltaTableState::with_version(version); for line in reader.lines() { - let action: action::Action = serde_json::from_str(line?.as_str())?; + let action: protocol::Action = serde_json::from_str(line?.as_str())?; new_state.process_action( action, table.config.require_tombstones, @@ -109,13 +112,13 @@ impl DeltaTableState { let preader = SerializedFileReader::new(data)?; let schema = preader.metadata().file_metadata().schema(); if !schema.is_group() { - return Err(DeltaTableError::from(action::ProtocolError::Generic( + return Err(DeltaTableError::from(protocol::ProtocolError::Generic( "Action record in checkpoint should be a struct".to_string(), ))); } for record in preader.get_row_iter(None)? { self.process_action( - action::Action::from_parquet_record(schema, &record.unwrap())?, + protocol::Action::from_parquet_record(schema, &record.unwrap())?, table_config.require_tombstones, table_config.require_files, )?; @@ -124,7 +127,7 @@ impl DeltaTableState { #[cfg(feature = "parquet2")] { - use crate::action::parquet2_read::actions_from_row_group; + use crate::protocol::parquet2_read::actions_from_row_group; use parquet2::read::read_metadata; let mut reader = std::io::Cursor::new(data); @@ -132,7 +135,7 @@ impl DeltaTableState { for row_group in metadata.row_groups { for action in actions_from_row_group(row_group, &mut reader) - .map_err(action::ProtocolError::from)? + .map_err(protocol::ProtocolError::from)? { self.process_action( action, @@ -164,7 +167,7 @@ impl DeltaTableState { } /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec { + pub fn commit_infos(&self) -> &Vec { &self.commit_infos } @@ -184,13 +187,13 @@ impl DeltaTableState { } /// Full list of tombstones (remove actions) representing files removed from table state). - pub fn all_tombstones(&self) -> &HashSet { + 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 { + pub fn unexpired_tombstones(&self) -> impl Iterator { let retention_timestamp = Utc::now().timestamp_millis() - self.tombstone_retention_millis; self.tombstones .iter() @@ -199,7 +202,7 @@ impl DeltaTableState { /// Full list of add actions representing all parquet files that are part of the current /// delta table state. - pub fn files(&self) -> &Vec { + pub fn files(&self) -> &Vec { self.files.as_ref() } @@ -228,6 +231,16 @@ impl DeltaTableState { self.min_writer_version } + /// Current supported reader features + pub fn reader_features(&self) -> Option<&HashSet> { + self.reader_features.as_ref() + } + + /// Current supported writer features + pub fn writer_features(&self) -> Option<&HashSet> { + self.writer_features.as_ref() + } + /// The most recent metadata of the table. pub fn current_metadata(&self) -> Option<&DeltaTableMetaData> { self.current_metadata.as_ref() @@ -289,6 +302,14 @@ impl DeltaTableState { self.min_writer_version = new_state.min_writer_version; } + if new_state.min_writer_version >= 5 { + self.writer_features = new_state.writer_features; + } + + if new_state.min_reader_version >= 3 { + self.reader_features = new_state.reader_features; + } + if new_state.current_metadata.is_some() { self.tombstone_retention_millis = new_state.tombstone_retention_millis; self.log_retention_millis = new_state.log_retention_millis; @@ -318,29 +339,30 @@ impl DeltaTableState { /// Process given action by updating current state. fn process_action( &mut self, - action: action::Action, + action: protocol::Action, require_tombstones: bool, require_files: bool, ) -> Result<(), ProtocolError> { match action { // TODO: optionally load CDC into TableState - action::Action::cdc(_v) => {} - action::Action::add(v) => { + protocol::Action::cdc(_v) => {} + protocol::Action::add(v) => { if require_files { - self.files.push(v.path_decoded()?); + self.files.push(v); } } - action::Action::remove(v) => { + protocol::Action::remove(v) => { if require_tombstones && require_files { - let v = v.path_decoded()?; self.tombstones.insert(v); } } - action::Action::protocol(v) => { + protocol::Action::protocol(v) => { self.min_reader_version = v.min_reader_version; self.min_writer_version = v.min_writer_version; + self.reader_features = v.reader_features; + self.writer_features = v.writer_features; } - action::Action::metaData(v) => { + protocol::Action::metaData(v) => { let md = DeltaTableMetaData::try_from(v)?; let table_config = TableConfig(&md.configuration); self.tombstone_retention_millis = @@ -350,16 +372,16 @@ impl DeltaTableState { self.enable_expired_log_cleanup = table_config.enable_expired_log_cleanup(); self.current_metadata = Some(md); } - action::Action::txn(v) => { + protocol::Action::txn(v) => { *self .app_transaction_version .entry(v.app_id) .or_insert(v.version) = v.version; } - action::Action::commitInfo(v) => { + protocol::Action::commitInfo(v) => { self.commit_infos.push(v); } - action::Action::domainMetadata(v) => { + protocol::Action::domainMetadata(v) => { self.domain_metadatas.push(v); } } @@ -370,13 +392,13 @@ impl DeltaTableState { /// Obtain Add actions for files that match the filter pub fn get_active_add_actions_by_partitions<'a>( &'a self, - filters: &'a [PartitionFilter<'a, &'a str>], - ) -> Result + '_, DeltaTableError> { + filters: &'a [PartitionFilter], + ) -> Result + '_, DeltaTableError> { let current_metadata = self.current_metadata().ok_or(DeltaTableError::NoMetadata)?; let nonpartitioned_columns: Vec = filters .iter() - .filter(|f| !current_metadata.partition_columns.contains(&f.key.into())) + .filter(|f| !current_metadata.partition_columns.contains(&f.key)) .map(|f| f.key.to_string()) .collect(); @@ -395,7 +417,7 @@ impl DeltaTableState { let partitions = add .partition_values .iter() - .map(|p| DeltaTablePartition::from_partition_value(p, "")) + .map(|p| DeltaTablePartition::from_partition_value((p.0, p.1), "")) .collect::>(); filters .iter() @@ -421,6 +443,8 @@ mod tests { app_transaction_version: Default::default(), min_reader_version: 0, min_writer_version: 0, + reader_features: None, + writer_features: None, current_metadata: None, tombstone_retention_millis: 0, log_retention_millis: 0, @@ -446,13 +470,15 @@ mod tests { current_metadata: None, min_reader_version: 1, min_writer_version: 1, + reader_features: None, + writer_features: None, app_transaction_version, tombstone_retention_millis: 0, log_retention_millis: 0, enable_expired_log_cleanup: true, }; - let txn_action = action::Action::txn(action::Txn { + let txn_action = protocol::Action::txn(protocol::Txn { app_id: "abc".to_string(), version: 2, last_updated: Some(0), diff --git a/rust/src/table_state_arrow.rs b/crates/deltalake-core/src/table/state_arrow.rs similarity index 99% rename from rust/src/table_state_arrow.rs rename to crates/deltalake-core/src/table/state_arrow.rs index d4765d48d8..34f858f415 100644 --- a/rust/src/table_state_arrow.rs +++ b/crates/deltalake-core/src/table/state_arrow.rs @@ -1,6 +1,6 @@ //! Methods to get Delta Table state in Arrow structures //! -//! See [crate::table_state::DeltaTableState]. +//! See [crate::table::DeltaTableState]. use std::borrow::Cow; use std::collections::{HashMap, HashSet, VecDeque}; @@ -16,9 +16,9 @@ use arrow_array::{ use arrow_schema::{DataType, Field, Fields, TimeUnit}; use itertools::Itertools; -use crate::action::{ColumnCountStat, ColumnValueStat, Stats}; +use super::state::DeltaTableState; use crate::errors::DeltaTableError; -use crate::table_state::DeltaTableState; +use crate::protocol::{ColumnCountStat, ColumnValueStat, Stats}; use crate::SchemaDataType; use crate::SchemaTypeStruct; diff --git a/rust/src/test_utils.rs b/crates/deltalake-core/src/test_utils.rs similarity index 99% rename from rust/src/test_utils.rs rename to crates/deltalake-core/src/test_utils.rs index 352a46d2b3..124ec0365b 100644 --- a/rust/src/test_utils.rs +++ b/crates/deltalake-core/src/test_utils.rs @@ -385,7 +385,7 @@ pub mod az_cli { /// small wrapper around s3 cli pub mod s3_cli { use super::set_env_if_not_set; - use crate::builder::s3_storage_options; + use crate::table::builder::s3_storage_options; use std::process::{Command, ExitStatus, Stdio}; /// Create a new bucket diff --git a/rust/src/writer/json.rs b/crates/deltalake-core/src/writer/json.rs similarity index 98% rename from rust/src/writer/json.rs rename to crates/deltalake-core/src/writer/json.rs index fc98cb0b90..f8d6d1a9e3 100644 --- a/rust/src/writer/json.rs +++ b/crates/deltalake-core/src/writer/json.rs @@ -3,16 +3,6 @@ use std::collections::HashMap; use std::convert::TryFrom; use std::sync::Arc; -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, PartitionPath, -}; -use super::{DeltaWriter, DeltaWriterError}; -use crate::builder::DeltaTableBuilder; -use crate::{action::Add, DeltaTable, DeltaTableError, DeltaTableMetaData, Schema}; -use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; - use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use arrow::record_batch::*; use bytes::Bytes; @@ -26,6 +16,18 @@ use parquet::{ use serde_json::Value; 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, +}; +use super::{utils::PartitionPath, DeltaWriter, DeltaWriterError}; +use crate::errors::DeltaTableError; +use crate::table::builder::DeltaTableBuilder; +use crate::table::DeltaTableMetaData; +use crate::{protocol::Add, DeltaTable, Schema}; +use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; + type BadValue = (Value, ParquetError); /// Writes messages to a delta lake table. diff --git a/rust/src/writer/mod.rs b/crates/deltalake-core/src/writer/mod.rs similarity index 98% rename from rust/src/writer/mod.rs rename to crates/deltalake-core/src/writer/mod.rs index 5685a71d48..8c5512127f 100644 --- a/rust/src/writer/mod.rs +++ b/crates/deltalake-core/src/writer/mod.rs @@ -7,9 +7,9 @@ use object_store::Error as ObjectStoreError; use parquet::errors::ParquetError; use serde_json::Value; -use crate::action::{Action, Add, ColumnCountStat, DeltaOperation, SaveMode}; use crate::errors::DeltaTableError; use crate::operations::transaction::commit; +use crate::protocol::{Action, Add, ColumnCountStat, DeltaOperation, SaveMode}; use crate::DeltaTable; pub use json::JsonWriter; diff --git a/rust/src/writer/record_batch.rs b/crates/deltalake-core/src/writer/record_batch.rs similarity index 95% rename from rust/src/writer/record_batch.rs rename to crates/deltalake-core/src/writer/record_batch.rs index 986b7b4026..a6486ae109 100644 --- a/rust/src/writer/record_batch.rs +++ b/crates/deltalake-core/src/writer/record_batch.rs @@ -4,37 +4,15 @@ //! Each Parquet file is buffered in-memory and only written once `flush()` is called on //! the writer. Once written, add actions are returned by the writer. It's the users responsibility //! to create the transaction using those actions. -//! -//! # Examples -//! -//! Write to an existing Delta Lake table: -//! ```rust ignore -//! let table = DeltaTable::try_from_uri("../path/to/table") -//! let batch: RecordBatch = ... -//! let mut writer = RecordBatchWriter::for_table(table, /*storage_options=*/ HashMap::new()) -//! writer.write(batch)?; -//! let actions: Vec = writer.flush()?.iter() -//! .map(|add| Action::add(add.into())) -//! .collect(); -//! let mut transaction = table.create_transaction(Some(DeltaTransactionOptions::new(/*max_retry_attempts=*/3))); -//! transaction.add_actions(actions); -//! async { -//! transaction.commit(Some(DeltaOperation::Write { -//! SaveMode::Append, -//! partitionBy: Some(table.get_metadata().partition_columns), -//! predicate: None, -//! })) -//! } -//! ``` + use std::{collections::HashMap, sync::Arc}; use arrow::array::{Array, UInt32Array}; use arrow::compute::{partition, take}; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; -use arrow::error::ArrowError; 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 object_store::{path::Path, ObjectStore}; use parquet::{arrow::ArrowWriter, errors::ParquetError}; @@ -47,9 +25,10 @@ use super::utils::{ stringified_partition_value, PartitionPath, ShareableBuffer, }; use super::{DeltaWriter, DeltaWriterError}; -use crate::builder::DeltaTableBuilder; use crate::errors::DeltaTableError; -use crate::{action::Add, storage::DeltaObjectStore, DeltaTable, DeltaTableMetaData, Schema}; +use crate::table::builder::DeltaTableBuilder; +use crate::table::DeltaTableMetaData; +use crate::{protocol::Add, storage::DeltaObjectStore, DeltaTable, Schema}; /// Writes messages to a delta lake table. pub struct RecordBatchWriter { @@ -405,7 +384,7 @@ fn lexsort_to_indices(arrays: &[ArrayRef]) -> UInt32Array { .iter() .map(|a| SortField::new(a.data_type().clone())) .collect(); - let mut converter = RowConverter::new(fields).unwrap(); + let converter = RowConverter::new(fields).unwrap(); let rows = converter.convert_columns(arrays).unwrap(); let mut sort: Vec<_> = rows.iter().enumerate().collect(); sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); diff --git a/rust/src/writer/stats.rs b/crates/deltalake-core/src/writer/stats.rs similarity index 98% rename from rust/src/writer/stats.rs rename to crates/deltalake-core/src/writer/stats.rs index d96508d14f..6cd1961798 100644 --- a/rust/src/writer/stats.rs +++ b/crates/deltalake-core/src/writer/stats.rs @@ -11,7 +11,7 @@ use parquet::{ }; use super::*; -use crate::action::{Add, ColumnValueStat, Stats}; +use crate::protocol::{Add, ColumnValueStat, Stats}; /// Creates an [`Add`] log action struct. pub fn create_add( @@ -147,12 +147,8 @@ impl StatsScalar { (Statistics::Boolean(v), _) => Ok(Self::Boolean(get_stat!(v))), // Int32 can be date, decimal, or just int32 (Statistics::Int32(v), Some(LogicalType::Date)) => { - let date = chrono::NaiveDate::from_num_days_from_ce_opt(get_stat!(v)).ok_or( - DeltaWriterError::StatsParsingFailed { - debug_value: v.to_string(), - logical_type: Some(LogicalType::Date), - }, - )?; + let epoch_start = chrono::NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); // creating from epoch should be infallible + let date = epoch_start + chrono::Duration::days(get_stat!(v) as i64); Ok(Self::Date(date)) } (Statistics::Int32(v), Some(LogicalType::Decimal { scale, .. })) => { @@ -479,9 +475,9 @@ mod tests { use super::utils::record_batch_from_message; use super::*; use crate::{ - action::{ColumnCountStat, ColumnValueStat}, - builder::DeltaTableBuilder, errors::DeltaTableError, + protocol::{ColumnCountStat, ColumnValueStat}, + table::builder::DeltaTableBuilder, DeltaTable, }; use lazy_static::lazy_static; @@ -540,9 +536,9 @@ mod tests { Value::from(12340.0), ), ( - simple_parquet_stat!(Statistics::Int32, 737821), + simple_parquet_stat!(Statistics::Int32, 10561), Some(LogicalType::Date), - Value::from("2021-01-31"), + Value::from("1998-12-01"), ), ( simple_parquet_stat!(Statistics::Int64, 1641040496789123456), diff --git a/rust/src/writer/test_utils.rs b/crates/deltalake-core/src/writer/test_utils.rs similarity index 90% rename from rust/src/writer/test_utils.rs rename to crates/deltalake-core/src/writer/test_utils.rs index f519ebf720..0e2770759d 100644 --- a/rust/src/writer/test_utils.rs +++ b/crates/deltalake-core/src/writer/test_utils.rs @@ -1,3 +1,5 @@ +//! Utilities for writing unit tests + use std::collections::HashMap; use std::sync::Arc; @@ -5,10 +7,11 @@ use arrow::compute::take; use arrow_array::{Int32Array, Int64Array, RecordBatch, StringArray, StructArray, UInt32Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; -use crate::delta::DeltaTableMetaData; -use crate::operations::create::CreateBuilder; -use crate::schema::Schema; -use crate::{DeltaTable, DeltaTableBuilder, SchemaDataType, SchemaField, SchemaTypeStruct}; +use crate::operations::{create::CreateBuilder, DeltaOps}; +use crate::schema::{Schema, SchemaTypeStruct}; +use crate::table::DeltaTableMetaData; +use crate::writer::SaveMode; +use crate::{DeltaConfigKey, DeltaTable, DeltaTableBuilder, SchemaDataType, SchemaField}; pub type TestResult = Result<(), Box>; @@ -46,6 +49,14 @@ pub fn get_record_batch(part: Option, with_null: bool) -> RecordBatch { } } +pub async fn write_batch(table: DeltaTable, batch: RecordBatch) -> DeltaTable { + DeltaOps(table) + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::Append) + .await + .expect("Failed to append") +} + pub fn get_arrow_schema(part: &Option) -> Arc { match part { Some(key) if key.contains("/id=") => Arc::new(ArrowSchema::new(vec![Field::new( @@ -282,6 +293,19 @@ pub fn get_delta_schema_with_nested_struct() -> Schema { ]) } +pub async fn setup_table_with_configuration( + key: DeltaConfigKey, + value: Option>, +) -> DeltaTable { + let table_schema = get_delta_schema(); + DeltaOps::new_in_memory() + .create() + .with_columns(table_schema.get_fields().clone()) + .with_configuration_property(key, value) + .await + .expect("Failed to create table") +} + pub fn create_bare_table() -> DeltaTable { let table_dir = tempfile::tempdir().unwrap(); let table_path = table_dir.path(); diff --git a/rust/src/writer/utils.rs b/crates/deltalake-core/src/writer/utils.rs similarity index 94% rename from rust/src/writer/utils.rs rename to crates/deltalake-core/src/writer/utils.rs index a12809916f..cfc089c164 100644 --- a/rust/src/writer/utils.rs +++ b/crates/deltalake-core/src/writer/utils.rs @@ -16,10 +16,12 @@ use arrow::datatypes::{ 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; @@ -45,13 +47,13 @@ impl PartitionPath { let partition_value = partition_values .get(k) .ok_or_else(|| DeltaWriterError::MissingPartitionColumn(k.to_string()))?; - - let partition_value = partition_value - .as_deref() - .unwrap_or(NULL_PARTITION_VALUE_DATA_PATH); - let part = format!("{k}={partition_value}"); - - path_parts.push(part); + 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 { @@ -60,6 +62,30 @@ impl PartitionPath { } } +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 diff --git a/rust/tests/checkpoint_writer.rs b/crates/deltalake-core/tests/checkpoint_writer.rs similarity index 97% rename from rust/tests/checkpoint_writer.rs rename to crates/deltalake-core/tests/checkpoint_writer.rs index 999f07ff9c..6788346ef8 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/crates/deltalake-core/tests/checkpoint_writer.rs @@ -1,14 +1,13 @@ -#![deny(warnings)] - #[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::*; + use deltalake_core::*; use pretty_assertions::assert_eq; use std::fs; use std::path::{Path, PathBuf}; @@ -23,7 +22,7 @@ mod simple_checkpoint { cleanup_checkpoint_files(log_path.as_path()); // Load the delta table at version 5 - let mut table = deltalake::open_table_with_version(table_location, 5) + let mut table = deltalake_core::open_table_with_version(table_location, 5) .await .unwrap(); @@ -50,7 +49,7 @@ mod simple_checkpoint { assert_eq!(10, version); // delta table should load just fine with the checkpoint in place - let table_result = deltalake::open_table(table_location).await.unwrap(); + 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()); @@ -91,8 +90,8 @@ mod delete_expired_delta_log_in_checkpoint { use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; - use deltalake::delta_config::DeltaConfigKey; - use deltalake::*; + use deltalake_core::table::config::DeltaConfigKey; + use deltalake_core::*; use maplit::hashmap; #[tokio::test] @@ -212,9 +211,9 @@ mod checkpoints_with_tombstones { use super::*; use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; - use deltalake::action::*; - use deltalake::delta_config::DeltaConfigKey; - use deltalake::*; + use deltalake_core::protocol::*; + use deltalake_core::table::config::DeltaConfigKey; + use deltalake_core::*; use maplit::hashmap; use parquet::file::reader::{FileReader, SerializedFileReader}; use parquet::schema::types::Type; @@ -361,7 +360,6 @@ mod checkpoints_with_tombstones { .map(Action::remove) .chain(std::iter::once(Action::add(add.clone()))) .collect(); - let operation = DeltaOperation::Optimize { predicate: None, target_size: 1000000, diff --git a/rust/tests/command_filesystem_check.rs b/crates/deltalake-core/tests/command_filesystem_check.rs similarity index 92% rename from rust/tests/command_filesystem_check.rs rename to crates/deltalake-core/tests/command_filesystem_check.rs index ced317d990..4988f4e5d7 100644 --- a/rust/tests/command_filesystem_check.rs +++ b/crates/deltalake-core/tests/command_filesystem_check.rs @@ -1,10 +1,10 @@ #![cfg(feature = "integration_test")] -use deltalake::test_utils::{ +use deltalake_core::test_utils::{ set_env_if_not_set, IntegrationContext, StorageIntegration, TestResult, TestTables, }; -use deltalake::Path; -use deltalake::{errors::DeltaTableError, DeltaOps}; +use deltalake_core::Path; +use deltalake_core::{errors::DeltaTableError, DeltaOps}; use serial_test::serial; mod common; @@ -12,7 +12,7 @@ mod common; #[tokio::test] #[serial] async fn test_filesystem_check_local() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Local).await?) + test_filesystem_check(StorageIntegration::Local).await } #[cfg(any(feature = "s3", feature = "s3-native-tls"))] @@ -21,21 +21,21 @@ async fn test_filesystem_check_local() -> TestResult { 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"); - Ok(test_filesystem_check(StorageIntegration::Amazon).await?) + test_filesystem_check(StorageIntegration::Amazon).await } #[cfg(feature = "azure")] #[tokio::test] #[serial] async fn test_filesystem_check_azure() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Microsoft).await?) + test_filesystem_check(StorageIntegration::Microsoft).await } #[cfg(feature = "gcs")] #[tokio::test] #[serial] async fn test_filesystem_check_gcp() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Google).await?) + test_filesystem_check(StorageIntegration::Google).await } #[cfg(feature = "hdfs")] @@ -73,7 +73,7 @@ async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table.state.all_tombstones().get(file).unwrap(); - assert_eq!(remove.data_change, true); + assert!(remove.data_change); // An additonal run should return an empty list of orphaned actions let op = DeltaOps::from(table); @@ -114,7 +114,7 @@ async fn test_filesystem_check_partitioned() -> TestResult { assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table.state.all_tombstones().get(file).unwrap(); - assert_eq!(remove.data_change, true); + assert!(remove.data_change); Ok(()) } @@ -170,7 +170,7 @@ async fn test_filesystem_check_outdated() -> TestResult { if let Err(DeltaTableError::VersionAlreadyExists(version)) = res { assert!(version == 3); } else { - assert!(false); + panic!(); } Ok(()) diff --git a/rust/tests/command_optimize.rs b/crates/deltalake-core/tests/command_optimize.rs similarity index 97% rename from rust/tests/command_optimize.rs rename to crates/deltalake-core/tests/command_optimize.rs index 2d31e330a4..70d161d69e 100644 --- a/rust/tests/command_optimize.rs +++ b/crates/deltalake-core/tests/command_optimize.rs @@ -6,14 +6,16 @@ use std::{collections::HashMap, error::Error, sync::Arc}; use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use arrow_select::concat::concat_batches; -use deltalake::action::{Action, DeltaOperation, Remove}; -use deltalake::errors::DeltaTableError; -use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics, OptimizeType}; -use deltalake::operations::transaction::commit; -use deltalake::operations::DeltaOps; -use deltalake::storage::ObjectStoreRef; -use deltalake::writer::{DeltaWriter, RecordBatchWriter}; -use deltalake::{DeltaTable, PartitionFilter, Path, SchemaDataType, SchemaField}; +use deltalake_core::errors::DeltaTableError; +use deltalake_core::operations::optimize::{ + create_merge_plan, MetricDetails, Metrics, OptimizeType, +}; +use deltalake_core::operations::transaction::commit; +use deltalake_core::operations::DeltaOps; +use deltalake_core::protocol::{Action, DeltaOperation, Remove}; +use deltalake_core::storage::ObjectStoreRef; +use deltalake_core::writer::{DeltaWriter, RecordBatchWriter}; +use deltalake_core::{DeltaTable, PartitionFilter, Path, SchemaDataType, SchemaField}; use futures::TryStreamExt; use object_store::ObjectStore; use parquet::arrow::async_reader::ParquetObjectReader; @@ -276,7 +278,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { )?; let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); - let other_dt = deltalake::open_table(uri).await?; + let other_dt = deltalake_core::open_table(uri).await?; let add = &other_dt.get_state().files()[0]; let remove = Remove { path: add.path.clone(), @@ -346,7 +348,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { )?; let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); - let mut other_dt = deltalake::open_table(uri).await?; + let mut other_dt = deltalake_core::open_table(uri).await?; let mut writer = RecordBatchWriter::for_table(&other_dt)?; write( &mut writer, diff --git a/rust/tests/command_restore.rs b/crates/deltalake-core/tests/command_restore.rs similarity index 98% rename from rust/tests/command_restore.rs rename to crates/deltalake-core/tests/command_restore.rs index cda9d5a257..ac9a37d73b 100644 --- a/rust/tests/command_restore.rs +++ b/crates/deltalake-core/tests/command_restore.rs @@ -4,8 +4,8 @@ use arrow::datatypes::Schema as ArrowSchema; use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType, Field}; use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; -use deltalake::action::SaveMode; -use deltalake::{DeltaOps, DeltaTable, SchemaDataType, SchemaField}; +use deltalake_core::protocol::SaveMode; +use deltalake_core::{DeltaOps, DeltaTable, SchemaDataType, SchemaField}; use rand::Rng; use std::collections::HashMap; use std::error::Error; diff --git a/rust/tests/command_vacuum.rs b/crates/deltalake-core/tests/command_vacuum.rs similarity index 88% rename from rust/tests/command_vacuum.rs rename to crates/deltalake-core/tests/command_vacuum.rs index ce6c6071c7..f44e1f86b1 100644 --- a/rust/tests/command_vacuum.rs +++ b/crates/deltalake-core/tests/command_vacuum.rs @@ -1,14 +1,41 @@ use chrono::Duration; use common::clock::TestClock; -use common::schemas::{get_vacuum_underscore_schema, get_xy_date_schema}; use common::TestContext; -use deltalake::operations::vacuum::Clock; -use deltalake::operations::DeltaOps; +use deltalake_core::operations::vacuum::Clock; +use deltalake_core::operations::DeltaOps; +use deltalake_core::Schema; 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() -> Schema { + serde_json::from_value(json!({ + "type": "struct", + "fields": [ + {"name": "x", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "y", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "date", "type": "string", "nullable": false, "metadata": {}}, + ] + })) + .unwrap() +} + +/// Schema that contains a column prefiexed with _ +pub fn get_vacuum_underscore_schema() -> Schema { + serde_json::from_value::(json!({ + "type": "struct", + "fields": [ + {"name": "x", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "y", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "_date", "type": "string", "nullable": false, "metadata": {}}, + ] + })) + .unwrap() +} + #[tokio::test] // Validate vacuum works on a non-partitioned table async fn test_non_partitioned_table() { diff --git a/rust/tests/commit_info_format.rs b/crates/deltalake-core/tests/commit_info_format.rs similarity index 90% rename from rust/tests/commit_info_format.rs rename to crates/deltalake-core/tests/commit_info_format.rs index c50a40d818..ba7d80a726 100644 --- a/rust/tests/commit_info_format.rs +++ b/crates/deltalake-core/tests/commit_info_format.rs @@ -1,8 +1,8 @@ #![allow(dead_code)] mod fs_common; -use deltalake::action::{Action, DeltaOperation, SaveMode}; -use deltalake::operations::transaction::commit; +use deltalake_core::operations::transaction::commit; +use deltalake_core::protocol::{Action, DeltaOperation, SaveMode}; use serde_json::json; use std::error::Error; use tempdir::TempDir; diff --git a/rust/tests/common/adls.rs b/crates/deltalake-core/tests/common/adls.rs similarity index 100% rename from rust/tests/common/adls.rs rename to crates/deltalake-core/tests/common/adls.rs diff --git a/rust/tests/common/clock.rs b/crates/deltalake-core/tests/common/clock.rs similarity index 92% rename from rust/tests/common/clock.rs rename to crates/deltalake-core/tests/common/clock.rs index 9143837406..246d7c652c 100644 --- a/rust/tests/common/clock.rs +++ b/crates/deltalake-core/tests/common/clock.rs @@ -1,5 +1,5 @@ use chrono::{Duration, Utc}; -use deltalake::operations::vacuum::Clock; +use deltalake_core::operations::vacuum::Clock; use std::sync::{Arc, Mutex}; #[derive(Clone, Debug)] diff --git a/rust/tests/common/datafusion.rs b/crates/deltalake-core/tests/common/datafusion.rs similarity index 74% rename from rust/tests/common/datafusion.rs rename to crates/deltalake-core/tests/common/datafusion.rs index 1a439225b9..67c9ac442b 100644 --- a/rust/tests/common/datafusion.rs +++ b/crates/deltalake-core/tests/common/datafusion.rs @@ -1,16 +1,16 @@ use datafusion::execution::context::{SessionContext, SessionState}; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::prelude::SessionConfig; -use deltalake::delta_datafusion::DeltaTableFactory; +use deltalake_core::delta_datafusion::DeltaTableFactory; use std::sync::Arc; pub fn context_with_delta_table_factory() -> SessionContext { let cfg = RuntimeConfig::new(); let env = RuntimeEnv::new(cfg).unwrap(); let ses = SessionConfig::new(); - let mut state = SessionState::with_config_rt(ses, Arc::new(env)); + let mut state = SessionState::new_with_config_rt(ses, Arc::new(env)); state .table_factories_mut() .insert("DELTATABLE".to_string(), Arc::new(DeltaTableFactory {})); - SessionContext::with_state(state) + SessionContext::new_with_state(state) } diff --git a/rust/tests/common/hdfs.rs b/crates/deltalake-core/tests/common/hdfs.rs similarity index 100% rename from rust/tests/common/hdfs.rs rename to crates/deltalake-core/tests/common/hdfs.rs diff --git a/rust/tests/common/mod.rs b/crates/deltalake-core/tests/common/mod.rs similarity index 92% rename from rust/tests/common/mod.rs rename to crates/deltalake-core/tests/common/mod.rs index 2966b1a911..a53d8b7641 100644 --- a/rust/tests/common/mod.rs +++ b/crates/deltalake-core/tests/common/mod.rs @@ -1,12 +1,12 @@ #![allow(dead_code, unused_variables)] use bytes::Bytes; -use deltalake::action::{self, Add, DeltaOperation, Remove, SaveMode}; -use deltalake::operations::create::CreateBuilder; -use deltalake::operations::transaction::commit; -use deltalake::storage::DeltaObjectStore; -use deltalake::DeltaTableBuilder; -use deltalake::{DeltaTable, Schema}; +use deltalake_core::operations::create::CreateBuilder; +use deltalake_core::operations::transaction::commit; +use deltalake_core::protocol::{self, Add, DeltaOperation, Remove, SaveMode}; +use deltalake_core::storage::DeltaObjectStore; +use deltalake_core::DeltaTableBuilder; +use deltalake_core::{DeltaTable, Schema}; use object_store::{path::Path, ObjectStore}; use std::any::Any; use std::collections::HashMap; @@ -22,7 +22,6 @@ pub mod datafusion; pub mod hdfs; #[cfg(any(feature = "s3", feature = "s3-native-tls"))] pub mod s3; -pub mod schemas; #[derive(Default)] pub struct TestContext { @@ -141,7 +140,7 @@ pub async fn add_file( partition_by: None, predicate: None, }; - let actions = vec![action::Action::add(add)]; + let actions = vec![protocol::Action::add(add)]; commit( table.object_store().as_ref(), &actions, @@ -174,7 +173,7 @@ pub async fn remove_file( ..Default::default() }; let operation = DeltaOperation::Delete { predicate: None }; - let actions = vec![action::Action::remove(remove)]; + let actions = vec![protocol::Action::remove(remove)]; commit( table.object_store().as_ref(), &actions, diff --git a/rust/tests/common/s3.rs b/crates/deltalake-core/tests/common/s3.rs similarity index 97% rename from rust/tests/common/s3.rs rename to crates/deltalake-core/tests/common/s3.rs index cc8135e23f..1ba5d1cd15 100644 --- a/rust/tests/common/s3.rs +++ b/crates/deltalake-core/tests/common/s3.rs @@ -21,7 +21,7 @@ pub async fn setup_s3_context() -> TestContext { env::set_var("AWS_ACCESS_KEY_ID", "deltalake"); env::set_var("AWS_SECRET_ACCESS_KEY", "weloverust"); env::set_var("AWS_DEFAULT_REGION", ®ion); - env::set_var("AWS_STORAGE_ALLOW_HTTP", "TRUE"); + env::set_var("AWS_ALLOW_HTTP", "TRUE"); cli.create_bucket(bucket_name, &endpoint); cli.create_table( @@ -40,7 +40,7 @@ pub async fn setup_s3_context() -> TestContext { 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("AWS_STORAGE_ALLOW_HTTP".to_owned(), "TRUE".to_string()); + config.insert("AWS_ALLOW_HTTP".to_owned(), "TRUE".to_string()); TestContext { config, diff --git a/rust/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 b/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 similarity index 100% rename from rust/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 rename to crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-0 diff --git a/rust/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 b/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 similarity index 100% rename from rust/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 rename to crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-1 diff --git a/rust/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 b/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 similarity index 100% rename from rust/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 rename to crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/.s3-optimization-2 diff --git a/rust/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc similarity index 100% rename from rust/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc rename to crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.crc diff --git a/rust/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/COVID-19_NYT/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00000-a496f40c-e091-413a-85f9-b1b69d4b3b4e-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00001-9d9d980b-c500-4f0b-bb96-771a515fbccc-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00002-8826af84-73bd-49a6-a4b9-e39ffed9c15a-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00003-539aff30-2349-4b0d-9726-c18630c6ad90-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00004-1bb9c3e3-c5b0-4d60-8420-23261f58a5eb-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00005-4d47f8ff-94db-4d32-806c-781a1cf123d2-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00006-d0ec7722-b30c-4e1c-92cd-b4fe8d3bb954-c000.snappy.parquet diff --git a/rust/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet b/crates/deltalake-core/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet similarity index 100% rename from rust/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet rename to crates/deltalake-core/tests/data/COVID-19_NYT/part-00007-4582392f-9fc2-41b0-ba97-a74b3afc8239-c000.snappy.parquet diff --git a/rust/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet b/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet similarity index 100% rename from rust/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet rename to crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.checkpoint.parquet diff --git a/rust/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint b/crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint similarity index 100% rename from rust/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint rename to crates/deltalake-core/tests/data/checkpoint_with_partitions/_delta_log/_last_checkpoint diff --git a/rust/tests/data/checkpoints/_delta_log/.gitignore b/crates/deltalake-core/tests/data/checkpoints/_delta_log/.gitignore similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/.gitignore rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/.gitignore diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000003.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000003.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000003.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000004.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000004.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000004.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000004.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000005.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000005.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000005.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000005.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000006.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000006.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000006.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000006.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000007.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000007.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000007.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000007.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000008.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000008.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000008.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000008.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000009.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000009.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000009.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000009.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000010.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000010.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000010.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000010.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000011.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000011.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000011.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000011.json diff --git a/rust/tests/data/checkpoints/_delta_log/00000000000000000012.json b/crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000012.json similarity index 100% rename from rust/tests/data/checkpoints/_delta_log/00000000000000000012.json rename to crates/deltalake-core/tests/data/checkpoints/_delta_log/00000000000000000012.json diff --git a/rust/tests/data/checkpoints_tombstones/.gitignore b/crates/deltalake-core/tests/data/checkpoints_tombstones/.gitignore similarity index 100% rename from rust/tests/data/checkpoints_tombstones/.gitignore rename to crates/deltalake-core/tests/data/checkpoints_tombstones/.gitignore diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.checkpoint.parquet diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000005.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000006.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000007.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000008.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000009.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000010.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000011.json diff --git a/rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json b/crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json similarity index 100% rename from rust/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json rename to crates/deltalake-core/tests/data/checkpoints_vacuumed/_delta_log/00000000000000000012.json diff --git a/rust/tests/data/concurrent_workers/_delta_log/.gitignore b/crates/deltalake-core/tests/data/concurrent_workers/_delta_log/.gitignore similarity index 100% rename from rust/tests/data/concurrent_workers/_delta_log/.gitignore rename to crates/deltalake-core/tests/data/concurrent_workers/_delta_log/.gitignore diff --git a/rust/tests/data/concurrent_workers/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/concurrent_workers/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/concurrent_workers/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/concurrent_workers/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-0.2.0/.part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/.part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000000.json.c6b312ca-665d-46ab-93a9-9f87ad2baa92.tmp.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000001.json.641a776e-6e56-4423-a9b0-7efc9e58826a.tmp.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000002.json.e64807e6-437c-44c9-abd2-50e6514d236e.tmp.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/..00000000000000000003.json.b374eda7-fa09-48ce-b06c-56025163f6ae.tmp.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.._last_checkpoint.477ba875-7a14-4e57-9973-1349c21a152c.tmp.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/.00000000000000000003.checkpoint.parquet.crc diff --git a/rust/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet diff --git a/rust/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/00000000000000000003.json diff --git a/rust/tests/data/delta-0.2.0/_delta_log/_last_checkpoint b/crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/_last_checkpoint similarity index 100% rename from rust/tests/data/delta-0.2.0/_delta_log/_last_checkpoint rename to crates/deltalake-core/tests/data/delta-0.2.0/_delta_log/_last_checkpoint diff --git a/rust/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.2.0/part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.8-empty/.part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.8-empty/.part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/delta-0.8-empty/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8-empty/part-00000-b0cc5102-6177-4d60-80d3-b5d170011621-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8-empty/part-00007-02b8c308-e5a7-41a8-a653-cb5594582017-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.8.0-date/.part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8.0-date/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0-date/part-00000-d22c627d-9655-4153-9527-f8995620fa42-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8.0-null-partition/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet b/crates/deltalake-core/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 rust/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0-null-partition/k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8.0-null-partition/k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet b/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 similarity index 100% rename from rust/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/deltalake-core/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/rust/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8.0-numeric-partition/_delta_log/00000000000000000000.json diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8.0-partitioned/_delta_log/00000000000000000000.json diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8.0-special-partition/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.8.0-special-partition/x=A%2FA/.part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet.crc b/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 similarity index 100% rename from rust/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/deltalake-core/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/rust/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet b/crates/deltalake-core/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 rust/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8.0-special-partition/x=B%20B/.part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet.crc b/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 similarity index 100% rename from rust/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/deltalake-core/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/rust/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet b/crates/deltalake-core/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 rust/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0-special-partition/x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-0.8.0/.part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-0.8.0/.part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.8.0/.part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-0.8.0/.part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/delta-0.8.0/.part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet.crc diff --git a/rust/tests/data/delta-0.8.0/_change_data/.gitkeep b/crates/deltalake-core/tests/data/delta-0.8.0/_change_data/.gitkeep similarity index 100% rename from rust/tests/data/delta-0.8.0/_change_data/.gitkeep rename to crates/deltalake-core/tests/data/delta-0.8.0/_change_data/.gitkeep diff --git a/rust/tests/data/delta-0.8.0/_delta_index/.gitkeep b/crates/deltalake-core/tests/data/delta-0.8.0/_delta_index/.gitkeep similarity index 100% rename from rust/tests/data/delta-0.8.0/_delta_index/.gitkeep rename to crates/deltalake-core/tests/data/delta-0.8.0/_delta_index/.gitkeep diff --git a/rust/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/delta-0.8.0/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0/part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0/part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet diff --git a/rust/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-0.8.0/part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000000.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000001.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000002.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000003.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000004.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000005.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000006.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000007.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000008.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000009.json diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000010.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000011.json diff --git a/rust/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 similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.crc rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.crc diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/00000000000000000012.json diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/_delta_log/_last_checkpoint diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-1c2d1a32-02dc-484f-87ff-4328ea56045d-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-28925d3a-bdf2-411e-bca9-b067444cbcb0-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-6630b7c4-0aca-405b-be86-68a812f2e4c8-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-74151571-7ec6-4bd6-9293-b5daab2ce667-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-7a509247-4f58-4453-9202-51d75dee59af-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-8e0aefe1-6645-4601-ac29-68cba64023b5-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-b26ba634-874c-45b0-a7ff-2f0395a53966-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-c4c8caec-299d-42a4-b50c-5a4bf724c037-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-ce300400-58ff-4b8f-8ba9-49422fdf9f2e-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e1262b3e-2959-4910-aea9-4eaf92f0c68c-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-e8e3753f-e2f6-4c9f-98f9-8f3d346727ba-c000.snappy.parquet diff --git a/rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet b/crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet similarity index 100% rename from rust/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet rename to crates/deltalake-core/tests/data/delta-1.2.1-only-struct-stats/part-00000-f73ff835-0571-4d67-ac43-4fbf948bfb9b-c000.snappy.parquet diff --git a/rust/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc b/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc similarity index 100% rename from rust/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc rename to crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/.00000000000000000000.json.crc diff --git a/rust/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/delta-2.2.0-partitioned-types/_delta_log/00000000000000000000.json diff --git a/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/tests/data/delta-2.2.0-partitioned-types/c1=4/c2=c/part-00003-f525f459-34f9-46f5-82d6-d42121d883fd.c000.snappy.parquet b/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 similarity index 100% rename from rust/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/deltalake-core/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/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/tests/data/delta-2.2.0-partitioned-types/c1=5/c2=b/part-00007-4e73fa3b-2c88-424a-8051-f8b54328ffdb.c000.snappy.parquet b/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 similarity index 100% rename from rust/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/deltalake-core/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/rust/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/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 similarity index 100% rename from rust/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/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 diff --git a/rust/tests/data/delta-2.2.0-partitioned-types/c1=6/c2=a/part-00011-10619b10-b691-4fd0-acc4-2a9608499d7c.c000.snappy.parquet b/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 similarity index 100% rename from rust/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/deltalake-core/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/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..14a083ee45 --- /dev/null +++ b/crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1681798578373,"operation":"DLT REFRESH","operationParameters":{"pipelineId":"5c7b47e9-12d6-4986-a601-6716734281ce","updateId":"adf780f9-789f-4857-8d41-e4d9938b61d5"},"isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Databricks-Runtime/dlt:11.3-delta-pipelines-801d604-ff1aff6-f0f113d-custom-local","txnId":"a2ad05c8-4559-41bc-b7b1-a94773bd2286"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"ac0a0120-970e-4d8c-ae92-b5244b055d6e","format":{"provider":"parquet","options":{}},"partitionColumns":[],"configuration":{},"createdTime":1681798577757}} \ No newline at end of file diff --git a/crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..ed4bd9056a --- /dev/null +++ b/crates/deltalake-core/tests/data/delta-live-table/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1681798612850,"operation":"DLT SETUP","operationParameters":{"pipelineId":"5c7b47e9-12d6-4986-a601-6716734281ce","updateId":"adf780f9-789f-4857-8d41-e4d9938b61d5"},"readVersion":0,"isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Databricks-Runtime/dlt:11.3-delta-pipelines-801d604-ff1aff6-f0f113d-custom-local","txnId":"c1e3c149-d6b1-4a51-b7a6-89c328d14833"}} +{"protocol":{"minReaderVersion":2,"minWriterVersion":5}} +{"metaData":{"id":"ac0a0120-970e-4d8c-ae92-b5244b055d6e","name":"SnowflakeTest_Snowflake_DTL_SHERPA_USER_TABLE","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"sherpa_user_id\",\"type\":\"decimal(38,0)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"enabled\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_login\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"first_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"full_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"email\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"job_title\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"hire_date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"skypoint_delta_index\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"pipelines.pipelineId":"5c7b47e9-12d6-4986-a601-6716734281ce","pipelines.autoOptimize.managed":"false","pipelines.metastore.tableName":"automation_retailsandbox.SnowflakeTest_Snowflake_DTL_SHERPA_USER_TABLE"},"createdTime":1681798577757}} \ No newline at end of file diff --git a/rust/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc diff --git a/rust/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/golden/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc diff --git a/rust/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/golden/data-reader-array-primitives/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet b/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet similarity index 100% rename from rust/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet rename to crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet diff --git a/rust/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet b/crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet similarity index 100% rename from rust/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet rename to crates/deltalake-core/tests/data/golden/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet diff --git a/rust/tests/data/http_requests/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/http_requests/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/http_requests/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/http_requests/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/http_requests/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet b/crates/deltalake-core/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet similarity index 100% rename from rust/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet rename to crates/deltalake-core/tests/data/http_requests/date=2023-04-13/part-00000-e853fe2e-6f42-450c-8af1-4145b73a96c7-c000.snappy.parquet diff --git a/rust/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet b/crates/deltalake-core/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet similarity index 100% rename from rust/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet rename to crates/deltalake-core/tests/data/http_requests/date=2023-04-14/part-00000-731ab1b3-85a8-4bc3-92e5-96347fe3fd84-c000.snappy.parquet diff --git a/rust/tests/data/issue_1374/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/issue_1374/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet b/crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet similarity index 100% rename from rust/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet rename to crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.checkpoint.parquet diff --git a/rust/tests/data/issue_1374/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/issue_1374/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/issue_1374/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/issue_1374/_delta_log/_last_checkpoint b/crates/deltalake-core/tests/data/issue_1374/_delta_log/_last_checkpoint similarity index 100% rename from rust/tests/data/issue_1374/_delta_log/_last_checkpoint rename to crates/deltalake-core/tests/data/issue_1374/_delta_log/_last_checkpoint diff --git a/rust/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet b/crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd47d-c000.snappy.parquet diff --git a/rust/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet b/crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet similarity index 100% rename from rust/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet rename to crates/deltalake-core/tests/data/issue_1374/date=2023-05-24/part-00000-e2b01fc6-a906-4008-82df-e98efdcdd49c-c000.snappy.parquet diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_commit/.part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_commit/.part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_commit/.part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_commit/.part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_commit/_delta_log/.gitignore b/crates/deltalake-core/tests/data/simple_commit/_delta_log/.gitignore similarity index 100% rename from rust/tests/data/simple_commit/_delta_log/.gitignore rename to crates/deltalake-core/tests/data/simple_commit/_delta_log/.gitignore diff --git a/rust/tests/data/simple_commit/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/simple_commit/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/simple_commit/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/simple_commit/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_commit/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet diff --git a/rust/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_commit/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet diff --git a/rust/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_commit/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet diff --git a/rust/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_commit/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/simple_table/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc similarity index 100% rename from rust/tests/data/simple_table/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc diff --git a/rust/tests/data/simple_table/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/simple_table/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/simple_table/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/simple_table/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/simple_table/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/simple_table/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/simple_table/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000003.json similarity index 100% rename from rust/tests/data/simple_table/_delta_log/00000000000000000003.json rename to crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000003.json diff --git a/rust/tests/data/simple_table/_delta_log/00000000000000000004.json b/crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000004.json similarity index 100% rename from rust/tests/data/simple_table/_delta_log/00000000000000000004.json rename to crates/deltalake-core/tests/data/simple_table/_delta_log/00000000000000000004.json diff --git a/rust/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet diff --git a/rust/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet 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 new file mode 100644 index 0000000000..52512aa8c3 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..52512aa8c3 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..52512aa8c3 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..52512aa8c3 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..efef74f19f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3531a2012f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3c57308100 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..673f94d101 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..6dee6e1233 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..d55f40cbc0 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..0cd5190c2c Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3a4bed33c5 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..42651db57e Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..673f94d101 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..aa9bdb761f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3a0fbe2b5f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..551c8f5742 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..4c990a1391 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..0cd5190c2c Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..d55f40cbc0 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..551c8f5742 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..cb6d982fbd Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3531a2012f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..6dee6e1233 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..86c64edb20 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3a0fbe2b5f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..413fc06a45 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..aa9bdb761f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..47ce4c1985 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..3a4bed33c5 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..4c990a1391 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..bcfd8e1eee Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..b8335adf96 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..12395f193e Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..22a136f3ec Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet.crc 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 new file mode 100644 index 0000000000..213b253a30 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/.part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet.crc differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..952f4177e7 --- /dev/null +++ b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000000.json @@ -0,0 +1,9 @@ +{"commitInfo":{"timestamp":1587968586154,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":5,"minWriterVersion":7,"readerFeatures": ["columnMapping","blahabl","deletionVectors","timestampNtz","v2Checkpoint"],"writerFeatures": ["appendOnly","invariants", "checkConstraints", "changeDataFeed","generatedColumns","columnMapping","identityColumns","deletionVectors","rowTracking","timestampNtz","domainMetadata","v2Checkpoint","icebergCompatV1"]}} +{"metaData":{"id":"5fba94ed-9794-4965-ba6e-6ee3c0d22af9","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1587968585495}} +{"add":{"path":"part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet","partitionValues":{},"size":262,"modificationTime":1587968586000,"dataChange":true}} +{"add":{"path":"part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968586000,"dataChange":true}} +{"add":{"path":"part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968586000,"dataChange":true}} +{"add":{"path":"part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968586000,"dataChange":true}} +{"add":{"path":"part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968586000,"dataChange":true}} +{"add":{"path":"part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968586000,"dataChange":true}} diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..0af2ff5ac4 --- /dev/null +++ b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000001.json @@ -0,0 +1,27 @@ +{"commitInfo":{"timestamp":1587968596254,"operation":"MERGE","operationParameters":{"predicate":"(oldData.`id` = newData.`id`)"},"readVersion":0,"isBlindAppend":false}} +{"remove":{"path":"part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet","deletionTimestamp":1587968596250,"dataChange":true}} +{"remove":{"path":"part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet","deletionTimestamp":1587968596253,"dataChange":true}} +{"remove":{"path":"part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet","deletionTimestamp":1587968596253,"dataChange":true}} +{"remove":{"path":"part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet","deletionTimestamp":1587968596253,"dataChange":true}} +{"remove":{"path":"part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet","deletionTimestamp":1587968596253,"dataChange":true}} +{"add":{"path":"part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet","partitionValues":{},"size":262,"modificationTime":1587968595000,"dataChange":true}} +{"add":{"path":"part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} +{"add":{"path":"part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968596000,"dataChange":true}} diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..8eeebb28d1 --- /dev/null +++ b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000002.json @@ -0,0 +1,29 @@ +{"commitInfo":{"timestamp":1587968604143,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"readVersion":1,"isBlindAppend":false}} +{"add":{"path":"part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet","partitionValues":{},"size":262,"modificationTime":1587968602000,"dataChange":true}} +{"add":{"path":"part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968602000,"dataChange":true}} +{"add":{"path":"part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968602000,"dataChange":true}} +{"add":{"path":"part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968602000,"dataChange":true}} +{"add":{"path":"part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968602000,"dataChange":true}} +{"add":{"path":"part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968602000,"dataChange":true}} +{"remove":{"path":"part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} +{"remove":{"path":"part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet","deletionTimestamp":1587968604143,"dataChange":true}} diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..7c411a8f24 --- /dev/null +++ b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000003.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1587968614187,"operation":"UPDATE","operationParameters":{"predicate":"((id#697L % cast(2 as bigint)) = cast(0 as bigint))"},"readVersion":2,"isBlindAppend":false}} +{"remove":{"path":"part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet","deletionTimestamp":1587968614096,"dataChange":true}} +{"remove":{"path":"part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet","deletionTimestamp":1587968614096,"dataChange":true}} +{"add":{"path":"part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968614000,"dataChange":true}} +{"add":{"path":"part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet","partitionValues":{},"size":429,"modificationTime":1587968614000,"dataChange":true}} diff --git a/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000004.json b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..8911adfbb2 --- /dev/null +++ b/crates/deltalake-core/tests/data/simple_table_features/_delta_log/00000000000000000004.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1587968626537,"operation":"DELETE","operationParameters":{"predicate":"[\"((`id` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT))\"]"},"readVersion":3,"isBlindAppend":false}} +{"remove":{"path":"part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet","deletionTimestamp":1587968626536,"dataChange":true}} +{"remove":{"path":"part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet","deletionTimestamp":1587968626536,"dataChange":true}} +{"add":{"path":"part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet","partitionValues":{},"size":262,"modificationTime":1587968626000,"dataChange":true}} diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet new file mode 100644 index 0000000000..3706170963 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet new file mode 100644 index 0000000000..3706170963 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet new file mode 100644 index 0000000000..3706170963 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00000-a922ea3b-ffc2-4ca1-9074-a278c24c4449-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet new file mode 100644 index 0000000000..3706170963 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet new file mode 100644 index 0000000000..466ecb9558 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet new file mode 100644 index 0000000000..2964979ef6 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet new file mode 100644 index 0000000000..8f3aadf7bb Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet new file mode 100644 index 0000000000..4cb7aae463 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet new file mode 100644 index 0000000000..ebf0901460 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet new file mode 100644 index 0000000000..a1caecfbac Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet new file mode 100644 index 0000000000..1cd670d193 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet new file mode 100644 index 0000000000..3b91e5be30 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet new file mode 100644 index 0000000000..5770a94ab6 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00004-95c9bc2c-ac85-4581-b3cc-84502b0c314f-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet new file mode 100644 index 0000000000..4cb7aae463 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00005-94a0861b-6455-4bd9-a080-73e02491c643-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet new file mode 100644 index 0000000000..8a4536406d Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet new file mode 100644 index 0000000000..ec63649165 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet new file mode 100644 index 0000000000..533801e17f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet new file mode 100644 index 0000000000..ea2985c2bc Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet new file mode 100644 index 0000000000..1cd670d193 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00011-42f838f9-a911-40af-98f5-2fccfa1b123f-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet new file mode 100644 index 0000000000..a1caecfbac Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00045-332fe409-7705-45b1-8d34-a0018cf73b70-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet new file mode 100644 index 0000000000..533801e17f Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00049-d3095817-de74-49c1-a888-81565a40161d-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet new file mode 100644 index 0000000000..17a7a308f4 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00058-b462c4cb-0c48-4148-8475-e21d2a2935f8-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet new file mode 100644 index 0000000000..2964979ef6 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00068-90650739-6a8e-492b-9403-53e33b3778ac-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet new file mode 100644 index 0000000000..ebf0901460 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00069-c78b4dd8-f955-4643-816f-cbd30a3f8c1b-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet new file mode 100644 index 0000000000..df7f737162 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00077-2fcb1c7c-5390-48ee-93f6-0acf11199a0d-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet new file mode 100644 index 0000000000..ec63649165 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00107-3f6c2aa0-fc28-4f4c-be15-135e15b398f4-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet new file mode 100644 index 0000000000..b5e5edad5e Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00112-07fd790a-11dc-4fde-9acd-623e740be992-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet new file mode 100644 index 0000000000..8a4536406d Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00116-bc66759e-6381-4f34-8cd4-6688aad8585d-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet new file mode 100644 index 0000000000..960d2b6e47 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00121-d8bc3e53-d2f2-48ce-9909-78da7294ffbd-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet new file mode 100644 index 0000000000..3b91e5be30 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00128-b31c3b81-24da-4a90-a8b4-578c9e9a218d-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet new file mode 100644 index 0000000000..ea2985c2bc Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00140-e9b1971d-d708-43fb-b07f-975d2226b800-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet new file mode 100644 index 0000000000..25fe1a24f1 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00143-03ceb88e-5283-4193-aa43-993cdf937fd3-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet new file mode 100644 index 0000000000..42cf6c0f89 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00150-ec6643fc-4963-4871-9613-f5ad1940b689-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet new file mode 100644 index 0000000000..d972771366 Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00154-4630673a-5227-48fb-a03d-e356fcd1564a-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet new file mode 100644 index 0000000000..c3cf2f57ca Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet differ diff --git a/crates/deltalake-core/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet new file mode 100644 index 0000000000..be5bf31a8b Binary files /dev/null and b/crates/deltalake-core/tests/data/simple_table_features/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet differ diff --git a/rust/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_change_data/cdc-00000-a846ce80-2eec-484d-bef7-0e63557786ca.c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.crc diff --git a/rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc b/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.crc diff --git a/rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc b/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.crc diff --git a/rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/simple_table_with_cdc/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-7444aec4-710a-4a4c-8abe-3323499043e9.c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_cdc/part-00000-996384f7-3fc5-4a5f-9921-6e56269ec2c9-c000.snappy.parquet diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/.part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet.crc diff --git a/rust/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 similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000003.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000004.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000005.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000006.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000007.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000008.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000009.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/00000000000000000010.json diff --git a/rust/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/_delta_log/_last_checkpoint diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-136c36f5-639d-4e95-bb0f-15cde3fb14eb-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-1abe25d3-0da6-46c5-98c1-7a69872fd797-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3810fbe0-9892-431d-bcfd-7de5788dfe8d-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-3fa65c69-4e55-4b18-a195-5f1ae583e553-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-72ecc4d6-2e44-4df4-99e6-23f1ac2b7b7c-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-7d239c98-d74b-4b02-b3f6-9f256992c633-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-8e7dc8c1-337b-40b8-a411-46d4295da531-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-9afd9224-729f-4420-a05e-8032113a6568-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e93060ad-9c8c-4170-a9da-7c6f53f6406b-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-e9c6df9a-e585-4c70-bc1f-de9bd8ae025b-c000.snappy.parquet diff --git a/rust/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet b/crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet similarity index 100% rename from rust/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet rename to crates/deltalake-core/tests/data/simple_table_with_checkpoint/part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000001.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000002.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000003.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000004.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000005.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000006.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000007.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000008.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000009.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.checkpoint.parquet diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000010.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000011.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000012.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000013.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000014.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000015.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000016.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000017.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000018.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000019.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.checkpoint.parquet diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.crc diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/00000000000000000020.json diff --git a/rust/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint b/crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint similarity index 100% rename from rust/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint rename to crates/deltalake-core/tests/data/table_with_deletion_logs/_delta_log/_last_checkpoint diff --git a/rust/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin b/crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin similarity index 100% rename from rust/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin rename to crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_8e4ca8be-7615-43cf-bc06-5d131148683f.bin diff --git a/rust/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin b/crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin similarity index 100% rename from rust/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin rename to crates/deltalake-core/tests/data/table_with_deletion_logs/deletion_vector_a2084964-69d4-4e1e-95f5-9bbd6571d5c3.bin diff --git a/rust/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_deletion_logs/part-00000-cb251d5e-b665-437a-a9a7-fbfc5137c77d.c000.snappy.parquet diff --git a/rust/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc similarity index 100% rename from rust/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc rename to crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.crc diff --git a/rust/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/table_with_edge_timestamps/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00000-a9dd181d-61aa-491d-b3c9-3eea548de6cb-c000.snappy.parquet diff --git a/rust/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_edge_timestamps/part-00001-f804d355-db40-4e13-a624-ddd50ce7f5c4-c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 diff --git a/rust/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 diff --git a/rust/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 diff --git a/rust/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.crc diff --git a/rust/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00044-22c23f7f-2411-4d88-b78c-cebe430cdd47.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00089-b466c656-9b4a-41d6-ab41-f02007d1658c.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00134-34f9b771-c60a-4bd4-bdc0-cd25fcc951c6.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00179-76f56874-b389-409b-8a2d-18462928840e.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00223-24d8cffb-245d-4027-87d6-940fcf593a60.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00268-365db28b-f856-49e6-a25f-b0211cf95d20.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00313-c528546e-c8ab-425d-b49a-5afe731aaac8.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00358-5937ec73-64a5-44dd-a793-922e30c1b9df.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00403-6af19469-0fc5-4809-b02a-ddebda3966e8.c000.snappy.parquet diff --git a/rust/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet b/crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet similarity index 100% rename from rust/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet rename to crates/deltalake-core/tests/data/table_with_liquid_clustering/part-00447-1755ad02-9b47-4287-8333-92cb01a5124b.c000.snappy.parquet diff --git a/rust/tests/fs_common/mod.rs b/crates/deltalake-core/tests/fs_common/mod.rs similarity index 94% rename from rust/tests/fs_common/mod.rs rename to crates/deltalake-core/tests/fs_common/mod.rs index 566436ac9c..61227ca46b 100644 --- a/rust/tests/fs_common/mod.rs +++ b/crates/deltalake-core/tests/fs_common/mod.rs @@ -1,9 +1,9 @@ use chrono::Utc; -use deltalake::action::{Action, Add, DeltaOperation, Remove, SaveMode}; -use deltalake::operations::create::CreateBuilder; -use deltalake::operations::transaction::commit; -use deltalake::storage::{DeltaObjectStore, GetResult, ObjectStoreResult}; -use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; +use deltalake_core::operations::create::CreateBuilder; +use deltalake_core::operations::transaction::commit; +use deltalake_core::protocol::{Action, Add, DeltaOperation, Remove, SaveMode}; +use deltalake_core::storage::{DeltaObjectStore, GetResult, ObjectStoreResult}; +use deltalake_core::{DeltaTable, Schema, SchemaDataType, SchemaField}; use object_store::path::Path as StorePath; use object_store::ObjectStore; use serde_json::Value; @@ -141,8 +141,8 @@ impl std::fmt::Display for SlowStore { impl SlowStore { pub fn new( location: Url, - options: impl Into + Clone, - ) -> deltalake::DeltaResult { + options: impl Into + Clone, + ) -> deltalake_core::DeltaResult { Ok(Self { inner: DeltaObjectStore::try_new(location, options).unwrap(), }) diff --git a/rust/tests/integration_checkpoint.rs b/crates/deltalake-core/tests/integration_checkpoint.rs similarity index 94% rename from rust/tests/integration_checkpoint.rs rename to crates/deltalake-core/tests/integration_checkpoint.rs index c4361ac7bf..7b2f9ea026 100644 --- a/rust/tests/integration_checkpoint.rs +++ b/crates/deltalake-core/tests/integration_checkpoint.rs @@ -1,10 +1,12 @@ #![cfg(feature = "integration_test")] use chrono::Utc; -use deltalake::checkpoints::{cleanup_expired_logs_for, create_checkpoint}; -use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult}; -use deltalake::writer::{DeltaWriter, JsonWriter}; -use deltalake::{errors::DeltaResult, DeltaOps, DeltaTableBuilder, ObjectStore, SchemaDataType}; +use deltalake_core::checkpoints::{cleanup_expired_logs_for, create_checkpoint}; +use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult}; +use deltalake_core::writer::{DeltaWriter, JsonWriter}; +use deltalake_core::{ + errors::DeltaResult, DeltaOps, DeltaTableBuilder, ObjectStore, SchemaDataType, +}; use object_store::path::Path; use serde_json::json; use serial_test::serial; diff --git a/rust/tests/integration_concurrent_writes.rs b/crates/deltalake-core/tests/integration_concurrent_writes.rs similarity index 91% rename from rust/tests/integration_concurrent_writes.rs rename to crates/deltalake-core/tests/integration_concurrent_writes.rs index 314e9ce9a6..0a6470d5d0 100644 --- a/rust/tests/integration_concurrent_writes.rs +++ b/crates/deltalake-core/tests/integration_concurrent_writes.rs @@ -1,10 +1,10 @@ #![cfg(feature = "integration_test")] -use deltalake::action::{Action, Add, DeltaOperation, SaveMode}; -use deltalake::operations::transaction::commit; -use deltalake::operations::DeltaOps; -use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; -use deltalake::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; +use deltalake_core::operations::transaction::commit; +use deltalake_core::operations::DeltaOps; +use deltalake_core::protocol::{Action, Add, DeltaOperation, SaveMode}; +use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; +use deltalake_core::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; use std::collections::HashMap; use std::future::Future; use std::iter::FromIterator; @@ -103,12 +103,12 @@ where assert_eq!(map.len() as i64, WORKERS * COMMITS); // check that we have unique and ascending versions committed - let mut versions = Vec::from_iter(map.keys().map(|x| x.clone())); + let mut versions = Vec::from_iter(map.keys().copied()); versions.sort(); assert_eq!(versions, Vec::from_iter(1i64..=WORKERS * COMMITS)); // check that each file for each worker is committed as expected - let mut files = Vec::from_iter(map.values().map(|x| x.clone())); + let mut files = Vec::from_iter(map.values().cloned()); files.sort(); let mut expected = Vec::new(); for w in 0..WORKERS { diff --git a/rust/tests/integration_datafusion.rs b/crates/deltalake-core/tests/integration_datafusion.rs similarity index 98% rename from rust/tests/integration_datafusion.rs rename to crates/deltalake-core/tests/integration_datafusion.rs index 2489ab9371..4978ea2a11 100644 --- a/rust/tests/integration_datafusion.rs +++ b/crates/deltalake-core/tests/integration_datafusion.rs @@ -2,7 +2,7 @@ use arrow::array::Int64Array; use common::datafusion::context_with_delta_table_factory; -use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; +use deltalake_core::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; use serial_test::serial; use std::collections::{HashMap, HashSet}; @@ -31,12 +31,12 @@ use datafusion_proto::bytes::{ }; use url::Url; -use deltalake::action::SaveMode; -use deltalake::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; -use deltalake::operations::create::CreateBuilder; -use deltalake::storage::DeltaObjectStore; -use deltalake::writer::{DeltaWriter, RecordBatchWriter}; -use deltalake::{ +use deltalake_core::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; +use deltalake_core::operations::create::CreateBuilder; +use deltalake_core::protocol::SaveMode; +use deltalake_core::storage::DeltaObjectStore; +use deltalake_core::writer::{DeltaWriter, RecordBatchWriter}; +use deltalake_core::{ operations::{write::WriteBuilder, DeltaOps}, DeltaTable, DeltaTableError, Schema, SchemaDataType, SchemaField, }; @@ -51,7 +51,7 @@ mod local { #[tokio::test] #[serial] async fn test_datafusion_local() -> TestResult { - Ok(test_datafusion(StorageIntegration::Local).await?) + test_datafusion(StorageIntegration::Local).await } fn get_scanned_files(node: &dyn ExecutionPlan) -> HashSet