diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index e5a4af32b9..7a67adf249 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -14,7 +14,7 @@ jobs: format: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: @@ -30,11 +30,12 @@ jobs: matrix: os: - ubuntu-latest + - macos-11 - macOS-10.15 - windows-latest runs-on: ${{ matrix.os }} steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: @@ -55,11 +56,12 @@ jobs: matrix: os: - ubuntu-latest + - macos-11 - macOS-10.15 - windows-latest runs-on: ${{ matrix.os }} steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: @@ -70,22 +72,76 @@ jobs: - name: Run tests run: cargo test --verbose --features datafusion-ext,azure - s3_test: + integration_test: runs-on: ubuntu-latest + services: + # fake-gcs: + # image: fsouza/fake-gcs-server + # ports: + # - 4443:4443 + localstack: + image: localstack/localstack:0.14.4 + ports: + - 4566:4566 + azurite: + image: mcr.microsoft.com/azure-storage/azurite + ports: + - 10000:10000 + + container: + image: amd64/rust + env: + # Disable full debug symbol generation to speed up CI build and keep memory down + # "1" means line tables only, which is useful for panic tracebacks. + RUSTFLAGS: "-C debuginfo=1" + # https://github.com/rust-lang/cargo/issues/10280 + CARGO_NET_GIT_FETCH_WITH_CLI: "true" + RUST_BACKTRACE: "1" + # Run integration tests + TEST_INTEGRATION: 1 + AWS_DEFAULT_REGION: "us-east-1" + AWS_ACCESS_KEY_ID: test + AWS_SECRET_ACCESS_KEY: test + AWS_ENDPOINT_URL: http://localstack:4566 + EC2_METADATA_ENDPOINT: http://ec2-metadata:1338 + GOOGLE_ENDPOINT_URL: https://fake-gcs:4443/storage/v1/b + GOOGLE_USE_EMULATOR: "1" + AZURE_USE_EMULATOR: "1" + AZURITE_BLOB_STORAGE_URL: "http://azurite:10000" + OBJECT_STORE_BUCKET: test-bucket + AZURE_STORAGE_CONNECTION_STRING: "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite:10000/devstoreaccount1;QueueEndpoint=http://azurite:10001/devstoreaccount1;" + steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 + + # - name: Configure Fake GCS Server (GCP emulation) + # run: | + # curl --insecure -v -X POST --data-binary '{"name":"test-bucket"}' -H "Content-Type: application/json" "https://fake-gcs:4443/storage/v1/b" + # echo '{"gcs_base_url": "https://fake-gcs:4443", "disable_oauth": true, "client_email": "", "private_key": ""}' > "$GOOGLE_SERVICE_ACCOUNT" + - name: Setup LocalStack (AWS emulation) + run: | + cd /tmp + curl "https://awscli.amazonaws.com/awscli-exe-linux-x86_64.zip" -o "awscliv2.zip" + unzip awscliv2.zip + ./aws/install + - name: Configure Azurite (Azure emulation) + # the magical connection string is from + # https://docs.microsoft.com/en-us/azure/storage/common/storage-use-azurite?tabs=visual-studio#http-connection-strings + run: | + curl -sL https://aka.ms/InstallAzureCLIDeb | bash + - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: profile: default toolchain: stable override: true + - uses: Swatinem/rust-cache@v1 - - name: Setup localstack - run: docker-compose up setup + - name: Run tests run: | - cargo test --features s3,datafusion-ext + cargo test --features integration_test,azure,s3,datafusion-ext paruqet2_test: runs-on: ubuntu-latest diff --git a/.gitignore b/.gitignore index b518bd40c3..80c3e8979c 100644 --- a/.gitignore +++ b/.gitignore @@ -15,3 +15,5 @@ tlaplus/*.toolbox/*/[0-9]*-[0-9]*-[0-9]*-[0-9]*-[0-9]*-[0-9]*/ **/.python-version .coverage *.env +__azurite* +__blobstorage__ diff --git a/Cargo.lock b/Cargo.lock index d3ba81dd22..9a6dbb23dd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2,12 +2,6 @@ # It is not intended for manual editing. version = 3 -[[package]] -name = "RustyXML" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b5ace29ee3216de37c0546865ad08edef58b0f9e76838ed8959a84a990e58c5" - [[package]] name = "adler" version = "1.0.2" @@ -87,9 +81,9 @@ checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" [[package]] name = "arrow" -version = "18.0.0" +version = "20.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5f89d2bc04fa746ee395d20c4cbfa508e4cce5c00bae816f0fae434fcfb9853" +checksum = "c72a69495f06c8abb65b76a87be192a26fa724380d1f292d4e558a32afed9989" dependencies = [ "ahash", "bitflags", @@ -106,22 +100,13 @@ dependencies = [ "multiversion", "num", "pyo3", - "rand 0.8.5", "regex", + "regex-syntax", "serde", "serde_derive", "serde_json", ] -[[package]] -name = "async-lock" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e97a171d191782fba31bb902b14ad94e24a68145032b7eedf871ab0bc0d077b6" -dependencies = [ - "event-listener", -] - [[package]] name = "async-stream" version = "0.3.3" @@ -143,18 +128,6 @@ dependencies = [ "syn", ] -[[package]] -name = "async-timer" -version = "1.0.0-beta.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faacdfdccd10db54656717fddcd1a2ab6cd1ab16c0d6e7d89ec365b885fc9844" -dependencies = [ - "error-code", - "libc", - "wasm-bindgen", - "winapi", -] - [[package]] name = "async-trait" version = "0.1.57" @@ -183,105 +156,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" -[[package]] -name = "azure_core" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0e2c7582699a3af9cc8a7bc81259519d8afb8eded1090d4fcd86de3db0eace1" -dependencies = [ - "async-trait", - "base64", - "bytes", - "chrono", - "dyn-clone", - "futures", - "getrandom", - "http", - "log", - "oauth2", - "pin-project", - "rand 0.8.5", - "reqwest", - "rustc_version", - "serde", - "serde_derive", - "serde_json", - "url", - "uuid 1.1.2", -] - -[[package]] -name = "azure_identity" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a80434580cb2e2a1915b57fbd3655b513c4acf149cfbb85747f91649d48833ae" -dependencies = [ - "async-lock", - "async-timer", - "async-trait", - "azure_core", - "base64", - "chrono", - "fix-hidden-lifetime-bug", - "futures", - "http", - "log", - "oauth2", - "serde", - "serde_json", - "url", - "uuid 1.1.2", -] - -[[package]] -name = "azure_storage" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1a5bc29e999268e618c202f157291930d749f1e4de55e01ecfda3990dd37dc7" -dependencies = [ - "RustyXML", - "async-trait", - "azure_core", - "base64", - "bytes", - "chrono", - "futures", - "hmac 0.12.1", - "http", - "log", - "once_cell", - "serde", - "serde-xml-rs", - "serde_derive", - "serde_json", - "sha2 0.10.2", - "url", - "uuid 1.1.2", -] - -[[package]] -name = "azure_storage_datalake" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41971cdf60cf59647979ef373b02ecc156fa126e99730e1674b8595f46797462" -dependencies = [ - "async-trait", - "azure_core", - "azure_storage", - "base64", - "bytes", - "chrono", - "futures", - "http", - "log", - "serde", - "serde-xml-rs", - "serde_derive", - "serde_json", - "url", - "uuid 1.1.2", -] - [[package]] name = "base64" version = "0.13.0" @@ -436,9 +310,9 @@ dependencies = [ [[package]] name = "comfy-table" -version = "6.0.0" +version = "6.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "121d8a5b0346092c18a4b2fd6f620d7a06f0eb7ac0a45860939a0884bc579c56" +checksum = "85914173c2f558d61613bfbbf1911f14e630895087a7ed2fafc0f5319e1536e7" dependencies = [ "strum", "strum_macros", @@ -521,20 +395,6 @@ dependencies = [ "itertools", ] -[[package]] -name = "crossbeam" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2801af0d36612ae591caa9568261fddce32ce6e08a7275ea334a06a4ad021a2c" -dependencies = [ - "cfg-if", - "crossbeam-channel", - "crossbeam-deque", - "crossbeam-epoch", - "crossbeam-queue", - "crossbeam-utils", -] - [[package]] name = "crossbeam-channel" version = "0.5.6" @@ -570,16 +430,6 @@ dependencies = [ "scopeguard", ] -[[package]] -name = "crossbeam-queue" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd42583b04998a5363558e5f9291ee5a5ff6b49944332103f251e7479a82aa7" -dependencies = [ - "cfg-if", - "crossbeam-utils", -] - [[package]] name = "crossbeam-utils" version = "0.8.11" @@ -638,15 +488,6 @@ dependencies = [ "memchr", ] -[[package]] -name = "ct-logs" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1a816186fa68d9e426e3cb4ae4dff1fcd8e4a2c34b781bf7a822574a0d0aac8" -dependencies = [ - "sct 0.6.1", -] - [[package]] name = "ctor" version = "0.1.23" @@ -666,14 +507,14 @@ dependencies = [ "cfg-if", "hashbrown", "lock_api", - "parking_lot_core 0.9.3", + "parking_lot_core", ] [[package]] name = "datafusion" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54617e523e447c9a139fdf3682eeca8f909934bd28cdd0032ebd0ff9783775e1" +checksum = "430b3983c7164cb113f297f45b68a69893c212cb4b80a8aeb6a8069eb93f745e" dependencies = [ "ahash", "arrow", @@ -695,7 +536,7 @@ dependencies = [ "num_cpus", "object_store", "ordered-float 3.0.0", - "parking_lot 0.12.1", + "parking_lot", "parquet", "paste", "pin-project-lite", @@ -711,22 +552,23 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "794ca54d3b144038c36b7a31d64c9545abb2edbdda6da055e481fb8a13e4e33b" +checksum = "594210b4819cc786d1a3dc7b17ff4f9b0c6ee522bcd0a4a52f80a41fd38d53c4" dependencies = [ "arrow", "object_store", "ordered-float 3.0.0", "parquet", + "serde_json", "sqlparser", ] [[package]] name = "datafusion-expr" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0087a4e55a861c7040314f217672259304fd26b5f174a065867df6b4ac659896" +checksum = "b91d4a86776ce8f7fe5df34955481d6fe77876dd278bf13098d6a1bdd3c24fb8" dependencies = [ "ahash", "arrow", @@ -736,9 +578,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b822b1a9f4f9c953b142190229085e2856fa9ee52844aa86b40d55edd6e7cc38" +checksum = "360f86f7dc943ca8e0da39982febac0a0fc0329d7ee58ea046438c9fed6dfec8" dependencies = [ "arrow", "async-trait", @@ -752,9 +594,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2328a0e901a89c46391be9445e6e55b6dd8002d4d177e578b0c4a2486ef07cda" +checksum = "a465299f2eeb2741b33777b42f607fe56458e137d0d7b80f69be72e771a48b81" dependencies = [ "ahash", "arrow", @@ -777,9 +619,9 @@ dependencies = [ [[package]] name = "datafusion-row" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef6b51e6398ed6dcc5e072c16722b9838f472b0c0ffe25b5df536927cda6044f" +checksum = "959a42a1f35c8fa1b47698df6995ab5ae8477e81c9c42852476666aeac4f80b7" dependencies = [ "arrow", "datafusion-common", @@ -789,9 +631,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "10.0.0" +version = "11.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb9ae561d6c3dcd09d253ff28f71396b576fca05fe4d0f4fb0e75ee2fc951c72" +checksum = "c69404e8774fe2c7d64998e94d856f32d3a908f9dc7215ce01e09895f13b4b62" dependencies = [ "ahash", "arrow", @@ -806,27 +648,19 @@ dependencies = [ name = "deltalake" version = "0.4.1" dependencies = [ - "anyhow", "arrow", - "async-stream", "async-trait", - "azure_core", - "azure_identity", - "azure_storage", - "azure_storage_datalake", "bytes", "cfg-if", "chrono", "criterion", - "crossbeam", "datafusion", + "dotenv", "dynamodb_lock", "errno", + "fs_extra", "futures", "glibc_version 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", - "hyper", - "hyper-proxy", - "hyper-rustls 0.23.0", "lazy_static", "libc", "log", @@ -841,27 +675,21 @@ dependencies = [ "pretty_assertions", "rand 0.8.5", "regex", - "reqwest", "rusoto_core", "rusoto_credential", "rusoto_dynamodb", "rusoto_glue", - "rusoto_s3", "rusoto_sts", "serde", "serde_json", "serial_test", - "tame-gcs", - "tame-oauth", "tempdir", "tempfile", "thiserror", "tokio", - "tokio-stream", "url", "utime", "uuid 1.1.2", - "walkdir", ] [[package]] @@ -943,10 +771,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" [[package]] -name = "dyn-clone" -version = "1.0.9" +name = "dotenv" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f94fa09c2aeea5b8839e414b7b841bf429fd25b9c522116ac97ee87856d88b2" +checksum = "77c90badedccf4105eca100756a0b1289e191f6fcbdadd3cee1d2f614f97da8f" [[package]] name = "dynamodb_lock" @@ -1024,22 +852,6 @@ dependencies = [ "libc", ] -[[package]] -name = "error-code" -version = "2.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64f18991e7bf11e7ffee451b5318b5c1a73c52d0d0ada6e5a3017c8c1ced6a21" -dependencies = [ - "libc", - "str-buf", -] - -[[package]] -name = "event-listener" -version = "2.5.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" - [[package]] name = "fallible-streaming-iterator" version = "0.1.9" @@ -1055,26 +867,6 @@ dependencies = [ "instant", ] -[[package]] -name = "fix-hidden-lifetime-bug" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4ae9c2016a663983d4e40a9ff967d6dcac59819672f0b47f2b17574e99c33c8" -dependencies = [ - "fix-hidden-lifetime-bug-proc_macros", -] - -[[package]] -name = "fix-hidden-lifetime-bug-proc_macros" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4c81935e123ab0741c4c4f0d9b8377e5fb21d3de7e062fa4b1263b1fbcba1ea" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "flatbuffers" version = "2.1.2" @@ -1127,6 +919,12 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "fs_extra" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2022715d62ab30faffd124d40b76f4134a550a87792276512b18d63272333394" + [[package]] name = "fuchsia-cprng" version = "0.1.1" @@ -1135,9 +933,9 @@ checksum = "a06f77d526c1a601b7c4cdd98f54b5eaabffc14d5f2f0296febdc7f357c6d3ba" [[package]] name = "futures" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab30e97ab6aacfe635fad58f22c2bb06c8b685f7421eb1e064a729e2a5f481fa" +checksum = "7f21eda599937fba36daeb58a22e8f5cee2d14c4a17b5b7739c7c8e5e3b8230c" dependencies = [ "futures-channel", "futures-core", @@ -1150,9 +948,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bfc52cbddcfd745bf1740338492bb0bd83d76c67b445f91c5fb29fae29ecaa1" +checksum = "30bdd20c28fadd505d0fd6712cdfcb0d4b5648baf45faef7f852afb2399bb050" dependencies = [ "futures-core", "futures-sink", @@ -1160,15 +958,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2acedae88d38235936c3922476b10fced7b2b68136f5e3c03c2d5be348a1115" +checksum = "4e5aa3de05362c3fb88de6531e6296e85cde7739cccad4b9dfeeb7f6ebce56bf" [[package]] name = "futures-executor" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d11aa21b5b587a64682c0094c2bdd4df0076c5324961a40cc3abd7f37930528" +checksum = "9ff63c23854bee61b6e9cd331d523909f238fc7636290b96826e9cfa5faa00ab" dependencies = [ "futures-core", "futures-task", @@ -1177,15 +975,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93a66fc6d035a26a3ae255a6d2bca35eda63ae4c5512bef54449113f7a1228e5" +checksum = "bbf4d2a7a308fd4578637c0b17c7e1c7ba127b8f6ba00b29f717e9655d85eb68" [[package]] name = "futures-macro" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0db9cce532b0eae2ccf2766ab246f114b56b9cf6d445e00c2549fbc100ca045d" +checksum = "42cd15d1c7456c04dbdf7e88bcd69760d74f3a798d6444e16974b505b0e62f17" dependencies = [ "proc-macro2", "quote", @@ -1194,21 +992,21 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca0bae1fe9752cf7fd9b0064c674ae63f97b37bc714d745cbde0afb7ec4e6765" +checksum = "21b20ba5a92e727ba30e72834706623d94ac93a725410b6a6b6fbc1b07f7ba56" [[package]] name = "futures-task" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "842fc63b931f4056a24d59de13fb1272134ce261816e063e634ad0c15cdc5306" +checksum = "a6508c467c73851293f390476d4491cf4d227dbabcd4170f3bb6044959b294f1" [[package]] name = "futures-util" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0828a5471e340229c11c77ca80017937ce3c58cb788a17e5f1c2d5c485a9577" +checksum = "44fb6cb1be61cc1d2e43b262516aafcf63b241cffdb1d3fa115f91d9c7b09c90" dependencies = [ "futures-channel", "futures-core", @@ -1239,10 +1037,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4eb1a864a501629691edf6c15a593b7a51eebaa1e8468e9ddc623de7c9b58ec6" dependencies = [ "cfg-if", - "js-sys", "libc", "wasi 0.11.0+wasi-snapshot-preview1", - "wasm-bindgen", ] [[package]] @@ -1310,31 +1106,6 @@ dependencies = [ "ahash", ] -[[package]] -name = "headers" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cff78e5788be1e0ab65b04d306b2ed5092c815ec97ec70f4ebd5aee158aa55d" -dependencies = [ - "base64", - "bitflags", - "bytes", - "headers-core", - "http", - "httpdate", - "mime", - "sha-1", -] - -[[package]] -name = "headers-core" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7f66481bfee273957b1f20485a4ff3362987f85b2c236580d81b4eb7a326429" -dependencies = [ - "http", -] - [[package]] name = "heck" version = "0.4.0" @@ -1366,15 +1137,6 @@ dependencies = [ "digest 0.9.0", ] -[[package]] -name = "hmac" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" -dependencies = [ - "digest 0.10.3", -] - [[package]] name = "http" version = "0.2.8" @@ -1448,42 +1210,6 @@ dependencies = [ "want", ] -[[package]] -name = "hyper-proxy" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca815a891b24fdfb243fa3239c86154392b0953ee584aa1a2a1f66d20cbe75cc" -dependencies = [ - "bytes", - "futures", - "headers", - "http", - "hyper", - "hyper-rustls 0.22.1", - "rustls-native-certs 0.5.0", - "tokio", - "tokio-rustls 0.22.0", - "tower-service", - "webpki 0.21.4", -] - -[[package]] -name = "hyper-rustls" -version = "0.22.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f9f7a97316d44c0af9b0301e65010573a853a9fc97046d7331d7f6bc0fd5a64" -dependencies = [ - "ct-logs", - "futures-util", - "hyper", - "log", - "rustls 0.19.1", - "rustls-native-certs 0.5.0", - "tokio", - "tokio-rustls 0.22.0", - "webpki 0.21.4", -] - [[package]] name = "hyper-rustls" version = "0.23.0" @@ -1493,10 +1219,10 @@ dependencies = [ "http", "hyper", "log", - "rustls 0.20.6", - "rustls-native-certs 0.6.2", + "rustls", + "rustls-native-certs", "tokio", - "tokio-rustls 0.23.4", + "tokio-rustls", ] [[package]] @@ -1743,9 +1469,9 @@ checksum = "8371e4e5341c3a96db127eb2465ac681ced4c433e01dd0e938adbef26ba93ba5" [[package]] name = "lock_api" -version = "0.4.7" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327fa5b6a6940e4699ec49a9beae1ea4845c6bab9314e4f84ac68742139d8c53" +checksum = "9f80bf5aacaf25cbfc8210d1cfb718f2bf3b11c4c54e5afe36c236853a8ec390" dependencies = [ "autocfg", "scopeguard", @@ -1762,9 +1488,9 @@ dependencies = [ [[package]] name = "lz4" -version = "1.23.3" +version = "1.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4edcb94251b1c375c459e5abe9fb0168c1c826c3370172684844f8f3f8d1a885" +checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" dependencies = [ "libc", "lz4-sys", @@ -1772,9 +1498,9 @@ dependencies = [ [[package]] name = "lz4-sys" -version = "1.9.3" +version = "1.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7be8908e2ed6f31c02db8a9fa962f03e36c53fbfde437363eae3306b85d7e17" +checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" dependencies = [ "cc", "libc", @@ -1978,38 +1704,26 @@ dependencies = [ "libc", ] -[[package]] -name = "oauth2" -version = "4.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d62c436394991641b970a92e23e8eeb4eb9bca74af4f5badc53bcd568daadbd" -dependencies = [ - "base64", - "chrono", - "getrandom", - "http", - "rand 0.8.5", - "reqwest", - "serde", - "serde_json", - "serde_path_to_error", - "sha2 0.10.2", - "thiserror", - "url", -] - [[package]] name = "object_store" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "857af043f5d9f36ed4f71815857f79b841412dda1cf0ca5a29608874f6f038e2" +version = "0.4.0" +source = "git+https://github.com/apache/arrow-rs?rev=86446ea88b6fc3e11b47aeedb9eb24f4d69b2f13#86446ea88b6fc3e11b47aeedb9eb24f4d69b2f13" dependencies = [ "async-trait", + "base64", "bytes", "chrono", "futures", "itertools", + "parking_lot", "percent-encoding", + "quick-xml", + "rand 0.8.5", + "reqwest", + "ring", + "rustls-pemfile", + "serde", + "serde_json", "snafu", "tokio", "tracing", @@ -2117,17 +1831,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "parking_lot" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d17b78036a60663b797adeaee46f5c9dfebb86948d1255007a1d6be0271ff99" -dependencies = [ - "instant", - "lock_api", - "parking_lot_core 0.8.5", -] - [[package]] name = "parking_lot" version = "0.12.1" @@ -2135,21 +1838,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" dependencies = [ "lock_api", - "parking_lot_core 0.9.3", -] - -[[package]] -name = "parking_lot_core" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d76e8e1493bcac0d2766c42737f34458f1c8c50c0d23bcb24ea953affb273216" -dependencies = [ - "cfg-if", - "instant", - "libc", - "redox_syscall", - "smallvec", - "winapi", + "parking_lot_core", ] [[package]] @@ -2167,10 +1856,11 @@ dependencies = [ [[package]] name = "parquet" -version = "18.0.0" +version = "20.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65f61759af307fad711e7656c705218402a8a79b776c893c20fef96e8ffd2a7d" +checksum = "d0f0af698fcf8d1d9f2971766ebef25821ffe8c39c91837c276dcd97e075d950" dependencies = [ + "ahash", "arrow", "base64", "brotli", @@ -2179,11 +1869,13 @@ dependencies = [ "chrono", "flate2", "futures", + "hashbrown", "lz4", "num", "num-bigint", "parquet-format", "rand 0.8.5", + "seq-macro", "snap", "thrift", "tokio", @@ -2370,7 +2062,7 @@ dependencies = [ "cfg-if", "indoc", "libc", - "parking_lot 0.12.1", + "parking_lot", "pyo3-build-config", "pyo3-ffi", "pyo3-macros", @@ -2426,6 +2118,16 @@ version = "1.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a1d01941d82fa2ab50be1e79e6714289dd7cde78eba4c074bc5a4374f650dfe0" +[[package]] +name = "quick-xml" +version = "0.23.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9279fbdacaad3baf559d8cabe0acc3d06e30ea14931af31af79578ac0946decc" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quote" version = "1.0.21" @@ -2593,7 +2295,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls 0.23.0", + "hyper-rustls", "hyper-tls", "ipnet", "js-sys", @@ -2603,14 +2305,14 @@ dependencies = [ "native-tls", "percent-encoding", "pin-project-lite", - "rustls 0.20.6", + "rustls", "rustls-pemfile", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls 0.23.4", + "tokio-rustls", "tokio-util", "tower-service", "url", @@ -2649,7 +2351,7 @@ dependencies = [ "futures", "http", "hyper", - "hyper-rustls 0.23.0", + "hyper-rustls", "hyper-tls", "lazy_static", "log", @@ -2733,7 +2435,7 @@ dependencies = [ "digest 0.9.0", "futures", "hex", - "hmac 0.11.0", + "hmac", "http", "hyper", "log", @@ -2771,19 +2473,6 @@ dependencies = [ "semver", ] -[[package]] -name = "rustls" -version = "0.19.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35edb675feee39aec9c99fa5ff985081995a06d594114ae14cbe797ad7b7a6d7" -dependencies = [ - "base64", - "log", - "ring", - "sct 0.6.1", - "webpki 0.21.4", -] - [[package]] name = "rustls" version = "0.20.6" @@ -2792,20 +2481,8 @@ checksum = "5aab8ee6c7097ed6057f43c187a62418d0c05a4bd5f18b3571db50ee0f9ce033" dependencies = [ "log", "ring", - "sct 0.7.0", - "webpki 0.22.0", -] - -[[package]] -name = "rustls-native-certs" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a07b7c1885bd8ed3831c289b7870b13ef46fe0e856d288c30d9cc17d75a2092" -dependencies = [ - "openssl-probe", - "rustls 0.19.1", - "schannel", - "security-framework", + "sct", + "webpki", ] [[package]] @@ -2876,16 +2553,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" -[[package]] -name = "sct" -version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b362b83898e0e69f38515b82ee15aa80636befe47c3b6d3d89a911e78fc228ce" -dependencies = [ - "ring", - "untrusted", -] - [[package]] name = "sct" version = "0.7.0" @@ -2940,18 +2607,6 @@ dependencies = [ "serde_derive", ] -[[package]] -name = "serde-xml-rs" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65162e9059be2f6a3421ebbb4fef3e74b7d9e7c60c50a0e292c6239f19f1edfa" -dependencies = [ - "log", - "serde", - "thiserror", - "xml-rs", -] - [[package]] name = "serde_cbor" version = "0.11.2" @@ -2979,21 +2634,11 @@ version = "1.0.85" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e55a28e3aaef9d5ce0506d0a14dbba8054ddc7e499ef522dd8b26859ec9d4a44" dependencies = [ - "indexmap", "itoa 1.0.3", "ryu", "serde", ] -[[package]] -name = "serde_path_to_error" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "184c643044780f7ceb59104cef98a5a6f12cb2288a7bc701ab93a362b49fd47d" -dependencies = [ - "serde", -] - [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -3016,7 +2661,7 @@ dependencies = [ "futures", "lazy_static", "log", - "parking_lot 0.12.1", + "parking_lot", "serial_test_derive", ] @@ -3032,17 +2677,6 @@ dependencies = [ "syn", ] -[[package]] -name = "sha-1" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "028f48d513f9678cda28f6e4064755b3fbb2af6acd672f2c209b62323f7aea0f" -dependencies = [ - "cfg-if", - "cpufeatures", - "digest 0.10.3", -] - [[package]] name = "sha2" version = "0.9.9" @@ -3127,9 +2761,9 @@ checksum = "45456094d1983e2ee2a18fdfebce3189fa451699d0502cb8e3b49dba5ba41451" [[package]] name = "socket2" -version = "0.4.4" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66d72b759436ae32898a2af0a14218dbf55efde3feeb170eb623637db85ee1e0" +checksum = "10c98bba371b9b22a71a9414e420f92ddeb2369239af08200816169d5e2dd7aa" dependencies = [ "libc", "winapi", @@ -3143,9 +2777,9 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "sqlparser" -version = "0.18.0" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f531637a13132fa3d38c54d4cd8f115905e5dc3e72f6e77bd6160481f482e25d" +checksum = "30c67d4d5de027da1da5a4ed4623f09ab5131d808364279a5f5abee5de9b8db3" dependencies = [ "log", ] @@ -3156,12 +2790,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" -[[package]] -name = "str-buf" -version = "1.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e08d8363704e6c71fc928674353e6b7c23dcea9d82d7012c8faf2a3a025f8d0" - [[package]] name = "streaming-decompression" version = "0.1.2" @@ -3207,42 +2835,6 @@ dependencies = [ "unicode-ident", ] -[[package]] -name = "tame-gcs" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d20ec2d6525a66afebdff9e1d8ef143c9deae9a3b040c61d3cfa9ae6fda80060" -dependencies = [ - "base64", - "bytes", - "chrono", - "http", - "percent-encoding", - "serde", - "serde_json", - "serde_urlencoded", - "thiserror", - "url", -] - -[[package]] -name = "tame-oauth" -version = "0.4.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9435c9348e480fad0f2215d5602e2dfad03df8a6398c4e7ceaeaa42758f26a8a" -dependencies = [ - "base64", - "chrono", - "http", - "lock_api", - "parking_lot 0.11.2", - "ring", - "serde", - "serde_json", - "twox-hash", - "url", -] - [[package]] name = "target-lexicon" version = "0.12.4" @@ -3382,7 +2974,7 @@ dependencies = [ "mio", "num_cpus", "once_cell", - "parking_lot 0.12.1", + "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2", @@ -3411,26 +3003,15 @@ dependencies = [ "tokio", ] -[[package]] -name = "tokio-rustls" -version = "0.22.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc6844de72e57df1980054b38be3a9f4702aba4858be64dd700181a8a6d0e1b6" -dependencies = [ - "rustls 0.19.1", - "tokio", - "webpki 0.21.4", -] - [[package]] name = "tokio-rustls" version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ - "rustls 0.20.6", + "rustls", "tokio", - "webpki 0.22.0", + "webpki", ] [[package]] @@ -3524,16 +3105,6 @@ version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" -[[package]] -name = "twox-hash" -version = "1.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" -dependencies = [ - "cfg-if", - "static_assertions", -] - [[package]] name = "typenum" version = "1.15.0" @@ -3595,7 +3166,6 @@ dependencies = [ "idna", "matches", "percent-encoding", - "serde", ] [[package]] @@ -3749,16 +3319,6 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "webpki" -version = "0.21.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8e38c0608262c46d4a56202ebabdeb094cef7e560ca7a226c6bf055188aa4ea" -dependencies = [ - "ring", - "untrusted", -] - [[package]] name = "webpki" version = "0.22.0" @@ -3775,7 +3335,7 @@ version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1c760f0d366a6c24a02ed7816e23e691f5d92291f94d15e836006fd11b04daf" dependencies = [ - "webpki 0.22.0", + "webpki", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 6aee43c6bb..85707e4db8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -11,3 +11,6 @@ exclude = ["proofs", "delta-inspect"] [profile.dev] split-debuginfo = "unpacked" + +[patch.crates-io] +object_store = { git = "https://github.com/apache/arrow-rs", rev = "86446ea88b6fc3e11b47aeedb9eb24f4d69b2f13" } diff --git a/build/setup_localstack.sh b/build/setup_localstack.sh index 45dd6595c0..6a305ff04c 100755 --- a/build/setup_localstack.sh +++ b/build/setup_localstack.sh @@ -1,6 +1,6 @@ #!/bin/bash -export AWS_DEFAULT_REGION=us-east-2 +export AWS_DEFAULT_REGION=us-east-1 export AWS_ACCESS_KEY_ID=test export AWS_SECRET_ACCESS_KEY=test export ENDPOINT=http://localstack:4566 diff --git a/docker-compose.yml b/docker-compose.yml index 599a8f7a91..b3ba58f818 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -1,10 +1,10 @@ version: "3.9" services: localstack: - image: localstack/localstack:0.12.11 + image: localstack/localstack:0.14.4 ports: - - "4566:4566" - - "${PORT_WEB_UI-8080}:${PORT_WEB_UI-8080}" + - 4566:4566 + - 8080:8080 environment: - SERVICES=s3,dynamodb - DEBUG=1 @@ -15,16 +15,26 @@ services: healthcheck: test: [ "CMD", "curl", "-f", "http://localhost:4566/health" ] - setup: - image: localstack/localstack:0.12.11 - depends_on: - - localstack - entrypoint: "/bin/bash" - command: - - /setup_localstack.sh - volumes: - - "./build/setup_localstack.sh:/setup_localstack.sh" - - "./rust/tests/data/golden:/data/golden" - - "./rust/tests/data/simple_table:/data/simple_table" - - "./rust/tests/data/simple_commit:/data/simple_commit" - - "./rust/tests/data/concurrent_workers:/data/concurrent_workers" + fake-gcs: + image: fsouza/fake-gcs-server + ports: + - 4443:4443 + + azurite: + image: mcr.microsoft.com/azure-storage/azurite + ports: + - 10000:10000 + + # setup-localstack: + # image: localstack/localstack:0.14.4 + # depends_on: + # - localstack + # entrypoint: "/bin/bash" + # command: + # - /setup_localstack.sh + # volumes: + # - "./build/setup_localstack.sh:/setup_localstack.sh" + # - "./rust/tests/data/golden:/data/golden" + # - "./rust/tests/data/simple_table:/data/simple_table" + # - "./rust/tests/data/simple_commit:/data/simple_commit" + # - "./rust/tests/data/concurrent_workers:/data/concurrent_workers" diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 9a56e18d3a..c55302a973 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -4,6 +4,7 @@ from dataclasses import dataclass from datetime import date, datetime from decimal import Decimal +from pathlib import Path from typing import ( TYPE_CHECKING, Any, @@ -144,10 +145,10 @@ def write_deltalake( if isinstance(table_or_uri, str): table = try_get_deltatable(table_or_uri) - table_uri = table_or_uri + table_uri = str(Path(table_or_uri).absolute()) else: table = table_or_uri - table_uri = table_uri = table._table.table_uri() + table_uri = table._table.table_uri() __enforce_append_only(table=table, configuration=configuration, mode=mode) diff --git a/python/src/lib.rs b/python/src/lib.rs index 5cb025e297..ac59f1f034 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -6,19 +6,17 @@ extern crate pyo3; pub mod schema; use chrono::{DateTime, FixedOffset, Utc}; -use deltalake::action; -use deltalake::action::Action; -use deltalake::action::{ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats}; -use deltalake::arrow::datatypes::Schema as ArrowSchema; -use deltalake::get_backend_for_uri; +use deltalake::action::{ + self, Action, ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats, +}; +use deltalake::arrow::{self, datatypes::Schema as ArrowSchema}; +use deltalake::builder::DeltaTableBuilder; use deltalake::partitions::PartitionFilter; -use deltalake::storage; use deltalake::DeltaDataTypeLong; use deltalake::DeltaDataTypeTimestamp; use deltalake::DeltaTableMetaData; use deltalake::DeltaTransactionOptions; -use deltalake::Schema; -use deltalake::{arrow, StorageBackend}; +use deltalake::{ObjectStore, Path, Schema}; use pyo3::create_exception; use pyo3::exceptions::PyException; use pyo3::exceptions::PyValueError; @@ -52,11 +50,11 @@ impl PyDeltaTableError { PyDeltaTableError::new_err(err.to_string()) } - fn from_storage(err: deltalake::StorageError) -> pyo3::PyErr { + fn from_tokio(err: tokio::io::Error) -> pyo3::PyErr { PyDeltaTableError::new_err(err.to_string()) } - fn from_tokio(err: tokio::io::Error) -> pyo3::PyErr { + fn from_object_store(err: deltalake::ObjectStoreError) -> pyo3::PyErr { PyDeltaTableError::new_err(err.to_string()) } @@ -105,18 +103,15 @@ impl RawDeltaTable { version: Option, storage_options: Option>, ) -> PyResult { - let mut table = deltalake::DeltaTableBuilder::from_uri(table_uri) - .map_err(PyDeltaTableError::from_raw)?; + let mut builder = deltalake::DeltaTableBuilder::from_uri(table_uri); if let Some(storage_options) = storage_options { - let backend = deltalake::get_backend_for_uri_with_options(table_uri, storage_options) - .map_err(PyDeltaTableError::from_storage)?; - table = table.with_storage_backend(backend) + builder = builder.with_storage_options(storage_options) } if let Some(version) = version { - table = table.with_version(version) + builder = builder.with_version(version) } let table = rt()? - .block_on(table.load()) + .block_on(builder.load()) .map_err(PyDeltaTableError::from_raw)?; Ok(RawDeltaTable { _table: table }) } @@ -142,8 +137,8 @@ impl RawDeltaTable { Ok(table_uri) } - pub fn table_uri(&self) -> PyResult<&str> { - Ok(&self._table.table_uri) + pub fn table_uri(&self) -> PyResult { + Ok(self._table.table_uri()) } pub fn version(&self) -> PyResult { @@ -491,15 +486,23 @@ fn filestats_to_expression<'py>( #[pyclass] pub struct DeltaStorageFsBackend { - _storage: Arc, + _storage: Arc, +} + +impl DeltaStorageFsBackend { + async fn get_object(&self, path: &Path) -> Result, deltalake::ObjectStoreError> { + Ok(self._storage.get(path).await?.bytes().await?.into()) + } } #[pymethods] impl DeltaStorageFsBackend { #[new] fn new(table_uri: &str) -> PyResult { - let storage = - storage::get_backend_for_uri(table_uri).map_err(PyDeltaTableError::from_storage)?; + let storage = DeltaTableBuilder::from_uri(table_uri) + .build_storage() + .map_err(PyDeltaTableError::from_raw)? + .storage_backend(); Ok(Self { _storage: storage }) } @@ -508,23 +511,25 @@ impl DeltaStorageFsBackend { } fn head_obj<'py>(&mut self, py: Python<'py>, path: &str) -> PyResult<&'py PyTuple> { + let path = Path::from(path); let obj = rt()? - .block_on(self._storage.head_obj(path)) - .map_err(PyDeltaTableError::from_storage)?; + .block_on(self._storage.head(&path)) + .map_err(PyDeltaTableError::from_object_store)?; Ok(PyTuple::new( py, &[ - obj.path.into_py(py), - obj.modified.timestamp().to_string().into_py(py), + obj.location.to_string().into_py(py), + obj.last_modified.timestamp().to_string().into_py(py), obj.size.into_py(py), ], )) } fn get_obj<'py>(&mut self, py: Python<'py>, path: &str) -> PyResult<&'py PyBytes> { + let path = Path::from(path); let obj = rt()? - .block_on(self._storage.get_obj(path)) - .map_err(PyDeltaTableError::from_storage)?; + .block_on(self.get_object(&path)) + .map_err(PyDeltaTableError::from_object_store)?; Ok(PyBytes::new(py, &obj)) } } @@ -590,12 +595,9 @@ fn write_new_deltalake( description: Option, configuration: Option>>, ) -> PyResult<()> { - let mut table = deltalake::DeltaTable::new( - &table_uri, - get_backend_for_uri(&table_uri).map_err(PyDeltaTableError::from_storage)?, - deltalake::DeltaTableConfig::default(), - ) - .map_err(PyDeltaTableError::from_raw)?; + let mut table = DeltaTableBuilder::from_uri(table_uri) + .build() + .map_err(PyDeltaTableError::from_raw)?; let metadata = DeltaTableMetaData::new( name, diff --git a/python/tests/conftest.py b/python/tests/conftest.py index 0be74eeb50..4c37843ec8 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -18,7 +18,7 @@ def s3cred() -> None: @pytest.fixture() def s3_localstack(monkeypatch): - monkeypatch.setenv("AWS_REGION", "us-east-2") + monkeypatch.setenv("AWS_REGION", "us-east-1") monkeypatch.setenv("AWS_ACCESS_KEY_ID", "test") monkeypatch.setenv("AWS_SECRET_ACCESS_KEY", "test") monkeypatch.setenv("AWS_ENDPOINT_URL", "http://localhost:4566") diff --git a/python/tests/test_fs.py b/python/tests/test_fs.py index 0991aa096f..cbb2eaf0a5 100644 --- a/python/tests/test_fs.py +++ b/python/tests/test_fs.py @@ -8,7 +8,7 @@ def test_normalize_path(): - backend = DeltaStorageFsBackend("") + backend = DeltaStorageFsBackend(".") assert backend.normalize_path("s3://foo/bar") == "s3://foo/bar" assert backend.normalize_path("s3://foo/bar/") == "s3://foo/bar" assert backend.normalize_path("/foo/bar//") == "/foo/bar" diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 880a78bb19..bc239aa887 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -9,120 +9,74 @@ description = "Native Delta Lake implementation in Rust" edition = "2021" [dependencies] -libc = ">=0.2.90, <1" -errno = "0.2" -thiserror = "1" -serde = { version = "1", features = ["derive"] } -serde_json = "1" -tokio = { version = "1", features = ["fs", "macros", "rt", "io-util"] } -tokio-stream = { version = "0", features = ["fs"] } -futures = "0.3" +arrow = { version = "20", optional = true } +async-trait = "0.1" bytes = "1" -log = "0" -regex = "1" chrono = "0.4.22" -uuid = { version = "1", features = ["serde", "v4"] } +cfg-if = "1" +errno = "0.2" +futures = "0.3" lazy_static = "1" -percent-encoding = "2" +log = "0" +libc = ">=0.2.90, <1" num-bigint = "0.4" num-traits = "0.2.15" -object_store = "0.3.0" +object_store = "0.4.0" +parquet = { version = "20", optional = true } +parquet2 = { version = "0.16", optional = true } +parquet-format = "~4.0.0" +percent-encoding = "2" +serde = { version = "1", features = ["derive"] } +serde_json = "1" +thiserror = "1" +tokio = { version = "1", features = ["macros", "rt"] } +regex = "1" +uuid = { version = "1", features = ["serde", "v4"] } url = "2.2" -# HTTP Client -reqwest = { version = "0.11", default-features = false, features = [ - "rustls-tls", - "stream", -], optional = true } - -# Azure -azure_core = { version = "0.3", optional = true } -azure_storage = { version = "0.4", optional = true } -azure_storage_datalake = { version = "0.4", optional = true } -azure_identity = { version = "0.4", optional = true } - -# S3 +# S3 lock client rusoto_core = { version = "0.48", default-features = false, optional = true } rusoto_credential = { version = "0.48", optional = true } -rusoto_s3 = { version = "0.48", default-features = false, optional = true } rusoto_sts = { version = "0.48", default-features = false, optional = true } rusoto_dynamodb = { version = "0.48", default-features = false, optional = true } -maplit = { version = "1", optional = true } -hyper = { version = "0.14.20", default-features = false, optional = true } -hyper-rustls = { version = "0.23.0", default-features = false, optional = true, features = [ - "http2", - "rustls-native-certs", - "tokio-runtime", -] } -hyper-proxy = { version = "0.9.1", default-features = false, optional = true, features = [ - "rustls", -] } # Glue rusoto_glue = { version = "0.48", default-features = false, optional = true } -# GCS -tame-gcs = { version = "0.10.0", optional = true } -tame-oauth = { version = "0.4.0", features = ["gcp"], optional = true } -async-stream = { version = "0.3.2", default-features = true, optional = true } - -# High-level writer -parquet-format = "~4.0.0" - -arrow = { version = "18", optional = true } -parquet = { version = "18", optional = true } - -parquet2 = { version = "0.16", optional = true} - -crossbeam = { version = "0", optional = true } - -cfg-if = "1" -async-trait = "0.1" -walkdir = "2" +# NOTE dependencies only for integration tests +fs_extra = { version = "1.2.0", optional = true } +tempdir = { version = "0", optional = true } [dependencies.datafusion] -version = "10" +version = "11" optional = true [features] default = ["arrow", "parquet"] rust-dataframe-ext = [] datafusion-ext = ["datafusion"] -azure = [ - "azure_core", - "azure_storage", - "azure_storage_datalake", - "azure_identity", - "reqwest", -] - +azure = ["object_store/azure"] +gcs = ["object_store/gcp"] s3 = [ "rusoto_core/native-tls", "rusoto_credential", - "rusoto_s3/native-tls", "rusoto_sts/native-tls", "rusoto_dynamodb/native-tls", - "maplit", "dynamodb_lock/native-tls", - "hyper", - "hyper-rustls", - "hyper-proxy", + "object_store/aws", ] s3-rustls = [ "rusoto_core/rustls", "rusoto_credential", - "rusoto_s3/rustls", "rusoto_sts/rustls", "rusoto_dynamodb/rustls", - "maplit", "dynamodb_lock/rustls", - "hyper", - "hyper-rustls", - "hyper-proxy", + "object_store/aws", ] -gcs = ["async-stream", "tame-gcs", "tame-oauth", "reqwest"] glue = ["s3", "rusoto_glue"] python = ["arrow/pyarrow"] +# used only for integration testing +integration_test = ["fs_extra", "tempdir"] [build-dependencies] glibc_version = "0" @@ -134,15 +88,15 @@ default-features = false optional = true [dev-dependencies] -utime = "0.3" +criterion = "0" +dotenv = "*" +maplit = "1" +pretty_assertions = "1.2.1" +rand = "0.8" serial_test = "0" -pretty_assertions = "1" tempdir = "0" tempfile = "3" -maplit = { version = "1" } -anyhow = "1" -rand = "0.8" -criterion = "0" +utime = "0.3" [[bench]] name = "read_checkpoint" diff --git a/rust/examples/read_delta_table.rs b/rust/examples/read_delta_table.rs index 5934b1f16f..127bb38d58 100644 --- a/rust/examples/read_delta_table.rs +++ b/rust/examples/read_delta_table.rs @@ -1,8 +1,5 @@ -extern crate anyhow; -extern crate deltalake; - #[tokio::main(flavor = "current_thread")] -async fn main() -> anyhow::Result<()> { +async fn main() -> Result<(), deltalake::DeltaTableError> { let table_path = "./tests/data/delta-0.8.0"; let table = deltalake::open_table(table_path).await?; println!("{}", table); diff --git a/rust/src/builder.rs b/rust/src/builder.rs new file mode 100644 index 0000000000..6ed0d334e6 --- /dev/null +++ b/rust/src/builder.rs @@ -0,0 +1,609 @@ +//! Create or load DeltaTables + +use crate::delta::{DeltaTable, DeltaTableError}; +use crate::schema::DeltaDataTypeVersion; +use crate::storage::file::FileStorageBackend; +#[cfg(any(feature = "s3", feature = "s3-rustls"))] +use crate::storage::s3::{S3StorageBackend, S3StorageOptions}; +use crate::storage::DeltaObjectStore; +use chrono::{DateTime, FixedOffset, Utc}; +#[cfg(any(feature = "s3", feature = "s3-rustls"))] +use object_store::aws::AmazonS3Builder; +#[cfg(feature = "azure")] +use object_store::azure::MicrosoftAzureBuilder; +#[cfg(feature = "gcs")] +use object_store::gcp::GoogleCloudStorageBuilder; +use object_store::path::Path; +use object_store::{DynObjectStore, Error as ObjectStoreError, Result as ObjectStoreResult}; +use std::collections::HashMap; +use std::sync::Arc; +use url::Url; + +/// possible version specifications for loading a delta table +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum DeltaVersion { + /// load the newest version + Newest, + /// specify the version to load + Version(DeltaDataTypeVersion), + /// specify the timestamp in UTC + Timestamp(DateTime), +} + +impl Default for DeltaVersion { + fn default() -> Self { + DeltaVersion::Newest + } +} + +/// Configuration options for delta table +#[derive(Debug)] +pub struct DeltaTableConfig { + /// Indicates whether our use case requires tracking tombstones. + /// This defaults to `true` + /// + /// Read-only applications never require tombstones. Tombstones + /// are only required when writing checkpoints, so even many writers + /// may want to skip them. + pub require_tombstones: bool, + + /// Indicates whether DeltaTable should track files. + /// This defaults to `true` + /// + /// Some append-only applications might have no need of tracking any files. + /// Hence, DeltaTable will be loaded with significant memory reduction. + pub require_files: bool, +} + +impl Default for DeltaTableConfig { + fn default() -> Self { + Self { + require_tombstones: true, + require_files: true, + } + } +} + +/// Load-time delta table configuration options +#[derive(Debug)] +pub struct DeltaTableLoadOptions { + /// table root uri + pub table_uri: String, + /// backend to access storage system + pub storage_backend: Option<(Arc, Path)>, + /// specify the version we are going to load: a time stamp, a version, or just the newest + /// available version + pub version: DeltaVersion, + /// Indicates whether our use case requires tracking tombstones. + /// This defaults to `true` + /// + /// Read-only applications never require tombstones. Tombstones + /// are only required when writing checkpoints, so even many writers + /// may want to skip them. + pub require_tombstones: bool, + /// Indicates whether DeltaTable should track files. + /// This defaults to `true` + /// + /// Some append-only applications might have no need of tracking any files. + /// Hence, DeltaTable will be loaded with significant memory reduction. + pub require_files: bool, +} + +impl DeltaTableLoadOptions { + /// create default table load options for a table uri + pub fn new(table_uri: impl Into) -> Self { + Self { + table_uri: table_uri.into(), + storage_backend: None, + require_tombstones: true, + require_files: true, + version: DeltaVersion::default(), + } + } +} + +/// builder for configuring a delta table load. +#[derive(Debug)] +pub struct DeltaTableBuilder { + options: DeltaTableLoadOptions, + storage_options: Option>, + #[allow(unused_variables)] + allow_http: Option, +} + +impl DeltaTableBuilder { + /// Creates `DeltaTableBuilder` from table uri + pub fn from_uri(table_uri: impl AsRef) -> Self { + DeltaTableBuilder { + options: DeltaTableLoadOptions::new(table_uri.as_ref()), + storage_options: None, + allow_http: None, + } + } + + /// Sets `require_tombstones=false` to the builder + pub fn without_tombstones(mut self) -> Self { + self.options.require_tombstones = false; + self + } + + /// Sets `require_files=false` to the builder + pub fn without_files(mut self) -> Self { + self.options.require_files = false; + self + } + + /// Sets `version` to the builder + pub fn with_version(mut self, version: DeltaDataTypeVersion) -> Self { + self.options.version = DeltaVersion::Version(version); + self + } + + /// specify the timestamp given as ISO-8601/RFC-3339 timestamp + pub fn with_datestring(self, date_string: &str) -> Result { + let datetime = + DateTime::::from(DateTime::::parse_from_rfc3339(date_string)?); + Ok(self.with_timestamp(datetime)) + } + + /// specify a timestamp + pub fn with_timestamp(mut self, timestamp: DateTime) -> Self { + self.options.version = DeltaVersion::Timestamp(timestamp); + self + } + + /// Set the storage backend. + /// + /// `table_root` denotes the [object_store::path::Path] within the store to the root of the delta. + /// This is required since we cannot infer the relative location of the table from the `table_uri` + /// For non-standard object store implementations. + /// + /// If a backend is not provided then it is derived from `table_uri`. + pub fn with_storage_backend(mut self, storage: Arc, table_root: &Path) -> Self { + self.options.storage_backend = Some((storage, table_root.clone())); + self + } + + /// Set options used to initialize storage backend + /// + /// Options may be passed in the HashMap or set as environment variables. + /// + /// [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. + /// [azure_storage_options] describes the available options for the Azure backend. + /// [gcp_storage_options] describes the available options for the Google Cloud Platform backend. + pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { + self.storage_options = Some(storage_options); + self + } + + /// Allows unsecure connections via http. + /// + /// This setting is most useful for testing / development when connecting to emulated services. + pub fn with_allow_http(mut self, allow_http: bool) -> Self { + self.allow_http = Some(allow_http); + self + } + + /// Build a delta storage backend for the given config + pub fn build_storage(self) -> Result, DeltaTableError> { + let (storage, storage_url) = match self.options.storage_backend { + // Some(storage) => storage, + None => get_storage_backend( + &self.options.table_uri, + self.storage_options, + self.allow_http, + )?, + _ => todo!(), + }; + let object_store = Arc::new(DeltaObjectStore::new(storage_url, storage)); + Ok(object_store) + } + + /// Build the delta Table from specified options. + /// + /// This will not load the log, i.e. the table is not initialized. To get an initialized + /// table use the `load` function + pub fn build(self) -> Result { + let (storage, storage_url) = match self.options.storage_backend { + // Some(storage) => storage, + None => get_storage_backend( + &self.options.table_uri, + self.storage_options, + self.allow_http, + )?, + _ => todo!(), + }; + let config = DeltaTableConfig { + require_tombstones: self.options.require_tombstones, + require_files: self.options.require_files, + }; + let object_store = Arc::new(DeltaObjectStore::new(storage_url, storage)); + Ok(DeltaTable::new(object_store, config)) + } + + /// finally load the table + pub async fn load(self) -> Result { + let version = self.options.version.clone(); + let mut table = self.build()?; + match version { + DeltaVersion::Newest => table.load().await?, + DeltaVersion::Version(v) => table.load_version(v).await?, + DeltaVersion::Timestamp(ts) => table.load_with_datetime(ts).await?, + } + Ok(table) + } +} + +/// Well known storage services +pub enum StorageService { + /// Local filesystem storage + Local, + /// S3 compliant service + S3, + /// Azure blob service + Azure, + /// Google cloud storage + GCS, + /// Unrecognized service + Unknown, +} + +/// A parsed URL identifying a storage location +/// for more information on the supported expressions +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct StorageUrl { + /// A URL that identifies a file or directory to list files from + pub(crate) url: Url, + /// The path prefix + pub(crate) prefix: Path, +} + +impl StorageUrl { + /// Parse a provided string as a `StorageUrl` + /// + /// # Paths without a Scheme + /// + /// If no scheme is provided, or the string is an absolute filesystem path + /// as determined [`std::path::Path::is_absolute`], the string will be + /// interpreted as a path on the local filesystem using the operating + /// system's standard path delimiter, i.e. `\` on Windows, `/` on Unix. + /// + /// Otherwise, the path will be resolved to an absolute path, returning + /// an error if it does not exist, and converted to a [file URI] + /// + /// If you wish to specify a path that does not exist on the local + /// machine you must provide it as a fully-qualified [file URI] + /// e.g. `file:///myfile.txt` + /// + /// [file URI]: https://en.wikipedia.org/wiki/File_URI_scheme + /// + /// # Well-known formats + /// + /// ## Azure + /// * az:/// + /// * abfs(s):/// + pub fn parse(s: impl AsRef) -> ObjectStoreResult { + let s = s.as_ref(); + + // This is necessary to handle the case of a path starting with a drive letter + if std::path::Path::new(s).is_absolute() { + return Self::parse_path(s); + } + + match Url::parse(s) { + Ok(url) => Ok(Self::new(url)), + Err(url::ParseError::RelativeUrlWithoutBase) => Self::parse_path(s), + Err(e) => Err(ObjectStoreError::Generic { + store: "DeltaObjectStore", + source: Box::new(e), + }), + } + } + + /// Creates a new [`StorageUrl`] interpreting `s` as a filesystem path + fn parse_path(s: &str) -> ObjectStoreResult { + let path = + std::path::Path::new(s) + .canonicalize() + .map_err(|e| ObjectStoreError::Generic { + store: "DeltaObjectStore", + source: Box::new(e), + })?; + let url = match path.is_file() { + true => Url::from_file_path(path).unwrap(), + false => Url::from_directory_path(path).unwrap(), + }; + + Ok(Self::new(url)) + } + + /// Creates a new [`StorageUrl`] from a url + fn new(url: Url) -> Self { + let prefix = Path::parse(url.path()).expect("should be URL safe"); + Self { url, prefix } + } + + /// Returns the URL scheme + pub fn scheme(&self) -> &str { + self.url.scheme() + } + + /// Returns this [`StorageUrl`] as a string + pub fn as_str(&self) -> &str { + self.as_ref() + } + + /// Returns the type of storage the URl refers to + pub fn service_type(&self) -> StorageService { + match self.url.scheme() { + "file" => StorageService::Local, + "az" | "abfs" | "abfss" | "adls2" | "azure" | "wasb" => StorageService::Azure, + // TODO is s3a permissible? + "s3" | "s3a" => StorageService::S3, + "gs" => StorageService::GCS, + _ => StorageService::Unknown, + } + } +} + +impl AsRef for StorageUrl { + fn as_ref(&self) -> &str { + self.url.as_ref() + } +} + +impl AsRef for StorageUrl { + fn as_ref(&self) -> &Url { + &self.url + } +} + +impl std::fmt::Display for StorageUrl { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.as_str().fmt(f) + } +} + +/// Create a new storage backend used in Delta table +fn get_storage_backend( + table_uri: impl AsRef, + // annotation needed for some feature builds + #[allow(unused_variables)] options: Option>, + #[allow(unused_variables)] allow_http: Option, +) -> ObjectStoreResult<(Arc, StorageUrl)> { + let storage_url = StorageUrl::parse(table_uri)?; + match storage_url.service_type() { + StorageService::Local => Ok((Arc::new(FileStorageBackend::new()), storage_url)), + #[cfg(any(feature = "s3", feature = "s3-rustls"))] + StorageService::S3 => { + let url: &Url = storage_url.as_ref(); + let bucket_name = url.host_str().ok_or(ObjectStoreError::NotImplemented)?; + let (mut builder, s3_options) = + get_s3_builder_from_options(options.unwrap_or_default()); + builder = builder.with_bucket_name(bucket_name); + if let Some(allow) = allow_http { + builder = builder.with_allow_http(allow); + } + Ok(( + Arc::new(S3StorageBackend::try_new( + Arc::new(builder.build()?), + s3_options, + )?), + storage_url, + )) + } + #[cfg(feature = "azure")] + StorageService::Azure => { + let url: &Url = storage_url.as_ref(); + // TODO we have to differentiate ... + let container_name = url.host_str().ok_or(ObjectStoreError::NotImplemented)?; + let mut builder = get_azure_builder_from_options(options.unwrap_or_default()) + .with_container_name(container_name); + if let Some(allow) = allow_http { + builder = builder.with_allow_http(allow); + } + Ok((Arc::new(builder.build()?), storage_url)) + } + #[cfg(feature = "gcs")] + StorageService::GCS => { + let url: &Url = storage_url.as_ref(); + let bucket_name = url.host_str().ok_or(ObjectStoreError::NotImplemented)?; + let builder = get_gcp_builder_from_options(options.unwrap_or_default()) + .with_bucket_name(bucket_name); + Ok((Arc::new(builder.build()?), storage_url)) + } + _ => todo!(), + } +} + +/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. +/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. +/// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. +pub mod s3_storage_options { + /// Custom S3 endpoint. + pub const AWS_ENDPOINT_URL: &str = "AWS_ENDPOINT_URL"; + /// The AWS region. + pub const AWS_REGION: &str = "AWS_REGION"; + /// The AWS_ACCESS_KEY_ID to use for S3. + pub const AWS_ACCESS_KEY_ID: &str = "AWS_ACCESS_KEY_ID"; + /// The AWS_SECRET_ACCESS_ID to use for S3. + pub const AWS_SECRET_ACCESS_KEY: &str = "AWS_SECRET_ACCESS_KEY"; + /// The AWS_SESSION_TOKEN to use for S3. + pub const AWS_SESSION_TOKEN: &str = "AWS_SESSION_TOKEN"; + /// Locking provider to use for safe atomic rename. + /// `dynamodb` is currently the only supported locking provider. + /// If not set, safe atomic rename is not available. + pub const AWS_S3_LOCKING_PROVIDER: &str = "AWS_S3_LOCKING_PROVIDER"; + /// The role to assume for S3 writes. + pub const AWS_S3_ASSUME_ROLE_ARN: &str = "AWS_S3_ASSUME_ROLE_ARN"; + /// The role session name to use when a role is assumed. If not provided a random session name is generated. + pub const AWS_S3_ROLE_SESSION_NAME: &str = "AWS_S3_ROLE_SESSION_NAME"; + /// The `pool_idle_timeout` option of aws http client. Has to be lower than 20 seconds, which is + /// default S3 server timeout . + /// However, since rusoto uses hyper as a client, its default timeout is 90 seconds + /// . + /// Hence, the `connection closed before message completed` could occur. + /// To avoid that, the default value of this setting is 15 seconds if it's not set otherwise. + pub const AWS_S3_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_S3_POOL_IDLE_TIMEOUT_SECONDS"; + /// The `pool_idle_timeout` for the as3_storage_optionsws sts client. See + /// the reasoning in `AWS_S3_POOL_IDLE_TIMEOUT_SECONDS`. + pub const AWS_STS_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_STS_POOL_IDLE_TIMEOUT_SECONDS"; + /// The number of retries for S3 GET requests failed with 500 Internal Server Error. + pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = + "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES"; + /// The web identity token file to use when using a web identity provider. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; + /// The role name to use for web identity. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; + /// The role session name to use for web identity. + /// NOTE: web identity related options are set in the environment when + /// creating an instance of [crate::storage::s3::S3StorageOptions]. + /// See also . + pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; + + /// The list of option keys owned by the S3 module. + /// Option keys not contained in this list will be added to the `extra_opts` + /// field of [crate::storage::s3::S3StorageOptions]. + /// `extra_opts` are passed to [dynamodb_lock::DynamoDbOptions] to configure the lock client. + pub const S3_OPTS: &[&str] = &[ + AWS_ENDPOINT_URL, + AWS_REGION, + AWS_ACCESS_KEY_ID, + AWS_SECRET_ACCESS_KEY, + AWS_SESSION_TOKEN, + AWS_S3_LOCKING_PROVIDER, + AWS_S3_ASSUME_ROLE_ARN, + AWS_S3_ROLE_SESSION_NAME, + AWS_WEB_IDENTITY_TOKEN_FILE, + AWS_ROLE_ARN, + AWS_ROLE_SESSION_NAME, + AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, + AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, + AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + ]; +} + +/// Generate a new AmazonS3Builder instance from a map of options +#[cfg(any(feature = "s3", feature = "s3-rustls"))] +pub fn get_s3_builder_from_options( + options: HashMap, +) -> (AmazonS3Builder, S3StorageOptions) { + let s3_options = S3StorageOptions::from_map(options); + + let mut builder = AmazonS3Builder::new(); + + if let Some(endpoint) = &s3_options.endpoint_url { + builder = builder.with_endpoint(endpoint); + } + builder = builder.with_region(s3_options.region.name()); + + if let Some(access_key_id) = &s3_options.aws_access_key_id { + builder = builder.with_access_key_id(access_key_id); + } + if let Some(secret_access_key) = &s3_options.aws_secret_access_key { + builder = builder.with_secret_access_key(secret_access_key); + } + if let Some(session_token) = &s3_options.aws_session_token { + builder = builder.with_token(session_token); + } + // TODO AWS_WEB_IDENTITY_TOKEN_FILE and AWS_ROLE_ARN are not configurable on the builder, but picked + // up by the build function if set on the environment. If we have them in the map, should we set them in the env? + // In the default case, always instance credentials are used. + (builder, s3_options) +} + +/// Storage option keys to use when creating azure storage backend. +/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. +pub mod azure_storage_options { + ///The ADLS Gen2 Access Key + pub const AZURE_STORAGE_ACCOUNT_KEY: &str = "AZURE_STORAGE_ACCOUNT_KEY"; + ///The name of storage account + pub const AZURE_STORAGE_ACCOUNT_NAME: &str = "AZURE_STORAGE_ACCOUNT_NAME"; + /// Connection string for connecting to azure storage account + pub const AZURE_STORAGE_CONNECTION_STRING: &str = "AZURE_STORAGE_CONNECTION_STRING"; + /// Service principal id + pub const AZURE_STORAGE_CLIENT_ID: &str = "AZURE_STORAGE_CLIENT_ID"; + /// Service principal secret + pub const AZURE_STORAGE_CLIENT_SECRET: &str = "AZURE_STORAGE_CLIENT_SECRET"; + /// ID for Azure (AAD) tenant where service principal is registered. + pub const AZURE_STORAGE_TENANT_ID: &str = "AZURE_STORAGE_TENANT_ID"; + /// Connect to a Azurite storage emulator instance + pub const AZURE_STORAGE_USE_EMULATOR: &str = "AZURE_STORAGE_USE_EMULATOR"; +} + +/// Generate a new MicrosoftAzureBuilder instance from a map of options +#[cfg(feature = "azure")] +pub fn get_azure_builder_from_options(options: HashMap) -> MicrosoftAzureBuilder { + let mut builder = MicrosoftAzureBuilder::new(); + + if let Some(account) = str_option(&options, azure_storage_options::AZURE_STORAGE_ACCOUNT_NAME) { + builder = builder.with_account(account); + } + if let Some(account_key) = + str_option(&options, azure_storage_options::AZURE_STORAGE_ACCOUNT_KEY) + { + builder = builder.with_access_key(account_key); + } + if let (Some(client_id), Some(client_secret), Some(tenant_id)) = ( + str_option(&options, azure_storage_options::AZURE_STORAGE_CLIENT_ID), + str_option(&options, azure_storage_options::AZURE_STORAGE_CLIENT_SECRET), + str_option(&options, azure_storage_options::AZURE_STORAGE_TENANT_ID), + ) { + builder = builder.with_client_secret_authorization(client_id, client_secret, tenant_id); + } + if let Some(_emulator) = str_option(&options, azure_storage_options::AZURE_STORAGE_USE_EMULATOR) + { + builder = builder.with_use_emulator(true).with_allow_http(true); + } + builder +} + +/// Storage option keys to use when creating gcp storage backend. +/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. +pub mod gcp_storage_options { + /// Path to the service account json file + pub const SERVICE_ACCOUNT: &str = "SERVICE_ACCOUNT"; + /// Path to the service account json file + pub const GOOGLE_SERVICE_ACCOUNT: &str = "GOOGLE_SERVICE_ACCOUNT"; + /// Configure google backend to ignore certificate errors for use with emulator. + pub const GOOGLE_USE_EMULATOR: &str = "GOOGLE_USE_EMULATOR"; +} + +/// Generate a new GoogleCloudStorageBuilder instance from a map of options +#[cfg(feature = "gcs")] +pub fn get_gcp_builder_from_options(options: HashMap) -> GoogleCloudStorageBuilder { + let mut builder = GoogleCloudStorageBuilder::new(); + + if let Some(account) = str_option(&options, gcp_storage_options::SERVICE_ACCOUNT) { + builder = builder.with_service_account_path(account); + } + + // TODO (roeap) We need either the option to insecure requests, or allow http connections + // to fake gcs, neither option is exposed by object store right now. + // #[cfg(test)] + // if let Ok(use_emulator) = std::env::var("GOOGLE_USE_EMULATOR") { + // use reqwest::Client; + // builder = builder.with_client( + // // ignore HTTPS errors in tests so we can use fake-gcs server + // Client::builder() + // .danger_accept_invalid_certs(true) + // .build() + // .expect("Error creating http client for testing"), + // ); + // } + + builder +} + +#[allow(dead_code)] +pub(crate) fn str_option(map: &HashMap, key: &str) -> Option { + map.get(key) + .map_or_else(|| std::env::var(key).ok(), |v| Some(v.to_owned())) +} diff --git a/rust/src/checkpoints.rs b/rust/src/checkpoints.rs index 3206b641cc..bc7f086961 100644 --- a/rust/src/checkpoints.rs +++ b/rust/src/checkpoints.rs @@ -19,9 +19,9 @@ use std::ops::Add; use super::action; use super::delta_arrow::delta_log_schema_for_table; -use super::object_store::DeltaObjectStore; use super::open_table_with_version; use super::schema::*; +use super::storage::DeltaObjectStore; use super::table_state::DeltaTableState; use super::time_utils; use super::DeltaTable; @@ -190,7 +190,8 @@ async fn flush_delete_files bool>( Ok(deleted_num) } -async fn cleanup_expired_logs_for( +/// exposed only for integration testing - DO NOT USE otherwise +pub async fn cleanup_expired_logs_for( until_version: DeltaDataTypeVersion, storage: &DeltaObjectStore, log_retention_timestamp: i64, @@ -574,9 +575,6 @@ mod tests { use super::*; use lazy_static::lazy_static; use serde_json::json; - use std::sync::Arc; - use std::time::Duration; - use uuid::Uuid; #[test] fn typed_partition_value_from_string_test() { @@ -838,87 +836,4 @@ mod tests { } }); } - - // Last-Modified for S3 could not be altered by user, hence using system pauses which makes - // test to run longer but reliable - async fn cleanup_metadata_test(table_path: &str) { - let object_store = - Arc::new(DeltaObjectStore::try_new_with_options(table_path, None).unwrap()); - - let log_path = |version| { - object_store - .log_path() - .child(format!("{:020}.json", version)) - }; - - // we don't need to actually populate files with content as cleanup works only with file's metadata - object_store - .put(&log_path(0), bytes::Bytes::from("")) - .await - .unwrap(); - - // since we cannot alter s3 object metadata, we mimic it with pauses - // also we forced to use 2 seconds since Last-Modified is stored in seconds - std::thread::sleep(Duration::from_secs(2)); - object_store - .put(&log_path(1), bytes::Bytes::from("")) - .await - .unwrap(); - - std::thread::sleep(Duration::from_secs(3)); - object_store - .put(&log_path(2), bytes::Bytes::from("")) - .await - .unwrap(); - - let v0time = object_store.head(&log_path(0)).await.unwrap().last_modified; - let v1time = object_store.head(&log_path(1)).await.unwrap().last_modified; - let v2time = object_store.head(&log_path(2)).await.unwrap().last_modified; - - // we choose the retention timestamp to be between v1 and v2 so v2 will be kept but other removed. - let retention_timestamp = - v1time.timestamp_millis() + (v2time.timestamp_millis() - v1time.timestamp_millis()) / 2; - - assert!(retention_timestamp > v0time.timestamp_millis()); - assert!(retention_timestamp > v1time.timestamp_millis()); - assert!(retention_timestamp < v2time.timestamp_millis()); - - let removed = crate::checkpoints::cleanup_expired_logs_for( - 3, - object_store.as_ref(), - retention_timestamp, - ) - .await - .unwrap(); - - assert_eq!(removed, 2); - assert!(object_store.head(&log_path(0)).await.is_err()); - assert!(object_store.head(&log_path(1)).await.is_err()); - assert!(object_store.head(&log_path(2)).await.is_ok()); - - // after test cleanup - object_store.delete(&log_path(2)).await.unwrap(); - } - - #[tokio::test] - async fn cleanup_metadata_fs_test() { - let table_path = format!("./tests/data/md_cleanup/{}", Uuid::new_v4()); - std::fs::create_dir_all(&table_path).unwrap(); - cleanup_metadata_test(&table_path).await; - std::fs::remove_dir_all(&table_path).unwrap(); - } - - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - mod cleanup_metadata_s3_test { - use super::*; - - #[tokio::test] - async fn cleanup_metadata_s3_test() { - std::env::set_var("AWS_ACCESS_KEY_ID", "test"); - std::env::set_var("AWS_SECRET_ACCESS_KEY", "test"); - std::env::set_var("AWS_ENDPOINT_URL", "http://localhost:4566"); - let table_path = format!("s3://deltars/md_cleanup/{}", Uuid::new_v4()); - cleanup_metadata_test(&table_path).await; - } - } } diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 250f4da871..58a4ea1d32 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -3,8 +3,17 @@ // Reference: https://github.com/delta-io/delta/blob/master/PROTOCOL.md // +use super::action; +use super::action::{Action, DeltaOperation}; +use super::partitions::{DeltaTablePartition, PartitionFilter}; +use super::schema::*; +use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; -use chrono::{DateTime, Duration, FixedOffset, Utc}; +pub use crate::builder::{DeltaTableBuilder, DeltaTableConfig, DeltaVersion}; +use crate::delta_config::DeltaConfigError; +use crate::storage::DeltaObjectStore; +use crate::vacuum::{Vacuum, VacuumError}; +use chrono::{DateTime, Duration, Utc}; use futures::StreamExt; use lazy_static::lazy_static; use log::debug; @@ -20,17 +29,6 @@ use std::sync::Arc; use std::{cmp::max, cmp::Ordering, collections::HashSet}; use uuid::Uuid; -use super::action; -use super::action::{Action, DeltaOperation}; -use super::partitions::{DeltaTablePartition, PartitionFilter}; -use super::schema::*; -use super::storage; -use super::storage::{StorageBackend, StorageError, UriError}; -use super::table_state::DeltaTableState; -use crate::delta_config::DeltaConfigError; -use crate::object_store::DeltaObjectStore; -use crate::vacuum::{Vacuum, VacuumError}; - /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] pub struct CheckPoint { @@ -78,13 +76,6 @@ pub enum DeltaTableError { }, /// Error returned when reading the delta log object failed. #[error("Failed to read delta log object: {}", .source)] - StorageError { - /// Storage error details when reading the delta log object failed. - #[from] - source: StorageError, - }, - /// Error returned when reading the delta log object failed. - #[error("Failed to read delta log object: {}", .source)] ObjectStore { /// Storage error details when reading the delta log object failed. #[from] @@ -98,14 +89,6 @@ pub enum DeltaTableError { #[from] source: parquet::errors::ParquetError, }, - /// Error returned when converting the schema in Arrow format failed. - #[cfg(feature = "arrow")] - #[error("Failed to convert into Arrow schema: {}", .source)] - ArrowError { - /// Arrow error details returned when converting the schema in Arrow format failed - #[from] - source: arrow::error::ArrowError, - }, /// Error returned when parsing checkpoint parquet using parquet2 crate. #[cfg(feature = "parquet2")] #[error("Failed to parse parquet: {}", .source)] @@ -114,12 +97,13 @@ pub enum DeltaTableError { #[from] source: parquet2::error::Error, }, - /// Error returned when the table has an invalid path. - #[error("Invalid table path: {}", .source)] - UriError { - /// Uri error details returned when the table has an invalid path. + /// Error returned when converting the schema in Arrow format failed. + #[cfg(feature = "arrow")] + #[error("Failed to convert into Arrow schema: {}", .source)] + ArrowError { + /// Arrow error details returned when converting the schema in Arrow format failed #[from] - source: UriError, + source: arrow::error::ArrowError, }, /// Error returned when the log record has an invalid JSON. #[error("Invalid JSON in log record: {}", .source)] @@ -404,164 +388,6 @@ impl From for LoadCheckpointError { } } -/// possible version specifications for loading a delta table -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum DeltaVersion { - /// load the newest version - Newest, - /// specify the version to load - Version(DeltaDataTypeVersion), - /// specify the timestamp in UTC - Timestamp(DateTime), -} - -impl Default for DeltaVersion { - fn default() -> Self { - DeltaVersion::Newest - } -} - -/// Configuration options for delta table -#[derive(Debug)] -pub struct DeltaTableConfig { - /// Indicates whether our use case requires tracking tombstones. - /// This defaults to `true` - /// - /// Read-only applications never require tombstones. Tombstones - /// are only required when writing checkpoints, so even many writers - /// may want to skip them. - pub require_tombstones: bool, - - /// Indicates whether DeltaTable should track files. - /// This defaults to `true` - /// - /// Some append-only applications might have no need of tracking any files. - /// Hence, DeltaTable will be loaded with significant memory reduction. - pub require_files: bool, -} - -impl Default for DeltaTableConfig { - fn default() -> Self { - Self { - require_tombstones: true, - require_files: true, - } - } -} - -/// Load-time delta table configuration options -#[derive(Debug)] -pub struct DeltaTableLoadOptions { - /// table root uri - pub table_uri: String, - /// backend to access storage system - pub storage_backend: Option>, - /// specify the version we are going to load: a time stamp, a version, or just the newest - /// available version - pub version: DeltaVersion, - /// Indicates whether our use case requires tracking tombstones. - /// This defaults to `true` - /// - /// Read-only applications never require tombstones. Tombstones - /// are only required when writing checkpoints, so even many writers - /// may want to skip them. - pub require_tombstones: bool, - /// Indicates whether DeltaTable should track files. - /// This defaults to `true` - /// - /// Some append-only applications might have no need of tracking any files. - /// Hence, DeltaTable will be loaded with significant memory reduction. - pub require_files: bool, -} - -impl DeltaTableLoadOptions { - /// create default table load options for a table uri - pub fn new(table_uri: &str) -> Result { - Ok(Self { - table_uri: table_uri.to_string(), - storage_backend: None, - require_tombstones: true, - require_files: true, - version: DeltaVersion::default(), - }) - } -} - -/// builder for configuring a delta table load. -#[derive(Debug)] -pub struct DeltaTableBuilder { - options: DeltaTableLoadOptions, -} - -impl DeltaTableBuilder { - /// Creates `DeltaTableBuilder` from table uri - pub fn from_uri(table_uri: &str) -> Result { - Ok(DeltaTableBuilder { - options: DeltaTableLoadOptions::new(table_uri)?, - }) - } - - /// Sets `require_tombstones=false` to the builder - pub fn without_tombstones(mut self) -> Self { - self.options.require_tombstones = false; - self - } - - /// Sets `require_files=false` to the builder - pub fn without_files(mut self) -> Self { - self.options.require_files = false; - self - } - - /// Sets `version` to the builder - pub fn with_version(mut self, version: DeltaDataTypeVersion) -> Self { - self.options.version = DeltaVersion::Version(version); - self - } - - /// specify the timestamp given as ISO-8601/RFC-3339 timestamp - pub fn with_datestring(self, date_string: &str) -> Result { - let datetime = - DateTime::::from(DateTime::::parse_from_rfc3339(date_string)?); - Ok(self.with_timestamp(datetime)) - } - - /// specify a timestamp - pub fn with_timestamp(mut self, timestamp: DateTime) -> Self { - self.options.version = DeltaVersion::Timestamp(timestamp); - self - } - - /// Set the storage backend. If a backend is not provided then it is derived from `table_uri` when `load` is called. - pub fn with_storage_backend(mut self, storage: Arc) -> Self { - self.options.storage_backend = Some(storage); - self - } - - /// finally load the table - pub async fn load(self) -> Result { - let storage = match self.options.storage_backend { - Some(storage) => storage, - None => storage::get_backend_for_uri(&self.options.table_uri)?, - }; - - let config = DeltaTableConfig { - require_tombstones: self.options.require_tombstones, - require_files: self.options.require_files, - }; - - let mut table = DeltaTable::new(self.options.table_uri, storage, config)?; - - match self.options.version { - DeltaVersion::Newest => table.load().await?, - DeltaVersion::Version(v) => table.load_version(v).await?, - DeltaVersion::Timestamp(ts) => table.load_with_datetime(ts).await?, - } - - Ok(table) - } -} - /// The next commit that's available from underlying storage /// TODO: Maybe remove this and replace it with Some/None and create a `Commit` struct to contain the next commit /// @@ -578,7 +404,7 @@ pub struct DeltaTable { /// The state of the table as of the most recent loaded Delta log entry. pub state: DeltaTableState, /// The URI the DeltaTable was loaded from. - pub table_uri: String, + // pub table_uri: String, /// the load options used during load pub config: DeltaTableConfig, // metadata @@ -590,11 +416,30 @@ pub struct DeltaTable { } impl DeltaTable { + /// Create a new Delta Table struct without loading any data from backing storage. + /// + /// NOTE: This is for advanced users. If you don't know why you need to use this method, please + /// call one of the `open_table` helper methods instead. + pub fn new(storage: Arc, config: DeltaTableConfig) -> Self { + Self { + state: DeltaTableState::with_version(-1), + storage, + config, + last_check_point: None, + version_timestamp: HashMap::new(), + } + } + /// get a shared reference to the delta object store pub fn object_store(&self) -> Arc { self.storage.clone() } + /// The + pub fn table_uri(&self) -> String { + self.storage.root_uri() + } + /// Return the uri of commit version. pub fn commit_uri_from_version(&self, version: DeltaDataTypeVersion) -> Path { let version = format!("{:020}.json", version); @@ -787,7 +632,7 @@ impl DeltaTable { if version < 0 { let err = format!( "No snapshot or version 0 found, perhaps {} is an empty dir?", - self.table_uri + self.table_uri() ); return Err(DeltaTableError::NotATable(err)); } @@ -886,7 +731,7 @@ impl DeltaTable { if self.version() == -1 { let err = format!( "No snapshot or version 0 found, perhaps {} is an empty dir?", - self.table_uri + self.table_uri() ); return Err(DeltaTableError::NotATable(err)); } @@ -992,7 +837,7 @@ impl DeltaTable { if version == -1 { let err = format!( "No snapshot or version 0 found, perhaps {} is an empty dir?", - self.table_uri + self.table_uri() ); return Err(DeltaTableError::NotATable(err)); } @@ -1225,27 +1070,6 @@ impl DeltaTable { Ok(version) } - /// Create a new Delta Table struct without loading any data from backing storage. - /// - /// NOTE: This is for advanced users. If you don't know why you need to use this method, please - /// call one of the `open_table` helper methods instead. - pub fn new( - table_uri: impl AsRef, - storage_backend: Arc, - config: DeltaTableConfig, - ) -> Result { - let storage = DeltaObjectStore::try_new(table_uri.as_ref(), storage_backend).unwrap(); - let root_uri = storage.root_uri(); - Ok(Self { - state: DeltaTableState::with_version(-1), - storage: Arc::new(storage), - table_uri: root_uri, - config, - last_check_point: None, - version_timestamp: HashMap::new(), - }) - } - /// Create a DeltaTable with version 0 given the provided MetaData, Protocol, and CommitInfo pub async fn create( &mut self, @@ -1337,7 +1161,7 @@ impl DeltaTable { impl fmt::Display for DeltaTable { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - writeln!(f, "DeltaTable({})", self.table_uri)?; + writeln!(f, "DeltaTable({})", self.table_uri())?; writeln!(f, "\tversion: {}", self.version())?; match self.state.current_metadata() { Some(metadata) => { @@ -1359,7 +1183,7 @@ impl fmt::Display for DeltaTable { impl std::fmt::Debug for DeltaTable { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - write!(f, "DeltaTable <{}>", self.table_uri) + write!(f, "DeltaTable <{}>", self.table_uri()) } } @@ -1572,8 +1396,7 @@ fn log_entry_from_actions(actions: &[Action]) -> Result Result { - let table = DeltaTableBuilder::from_uri(table_uri)?.load().await?; - + let table = DeltaTableBuilder::from_uri(table_uri).load().await?; Ok(table) } @@ -1583,7 +1406,7 @@ pub async fn open_table_with_version( table_uri: &str, version: DeltaDataTypeVersion, ) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri)? + let table = DeltaTableBuilder::from_uri(table_uri) .with_version(version) .load() .await?; @@ -1594,7 +1417,7 @@ pub async fn open_table_with_version( /// 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: &str, ds: &str) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri)? + let table = DeltaTableBuilder::from_uri(table_uri) .with_datestring(ds)? .load() .await?; @@ -1609,6 +1432,7 @@ pub fn crate_version() -> &'static str { #[cfg(test)] mod tests { use super::*; + use crate::builder::DeltaTableBuilder; use pretty_assertions::assert_eq; use std::io::{BufRead, BufReader}; use std::{collections::HashMap, fs::File, path::Path}; @@ -1623,9 +1447,8 @@ mod tests { ] .iter() { - let be = storage::get_backend_for_uri(table_uri).unwrap(); - let table = DeltaTable::new(table_uri, be, DeltaTableConfig::default()).unwrap(); - assert_eq!(table.table_uri, "s3://tests/data/delta-0.8.0"); + let table = DeltaTableBuilder::from_uri(table_uri).build().unwrap(); + assert_eq!(table.table_uri(), "s3://tests/data/delta-0.8.0"); } } @@ -1665,11 +1488,9 @@ mod tests { let table_dir = tmp_dir.path().join("test_create"); std::fs::create_dir(&table_dir).unwrap(); - let path = table_dir.to_str().unwrap(); - let backend = Arc::new(storage::file::FileStorageBackend::new( - tmp_dir.path().to_str().unwrap(), - )); - let mut dt = DeltaTable::new(path, backend, DeltaTableConfig::default()).unwrap(); + let mut dt = DeltaTableBuilder::from_uri(table_dir.to_str().unwrap()) + .build() + .unwrap(); let mut commit_info = Map::::new(); commit_info.insert( @@ -1691,7 +1512,8 @@ mod tests { assert_eq!(dt.state.files().len(), 0); // assert new _delta_log file created in tempDir - let table_path = Path::new(&dt.table_uri); + let table_uri = dt.table_uri(); + let table_path = Path::new(&table_uri); assert!(table_path.exists()); let delta_log = table_path.join("_delta_log"); diff --git a/rust/src/delta_arrow.rs b/rust/src/delta_arrow.rs index e572160959..45d2555509 100644 --- a/rust/src/delta_arrow.rs +++ b/rust/src/delta_arrow.rs @@ -117,7 +117,8 @@ impl TryFrom<&schema::SchemaDataType> for ArrowDataType { .get(2) .and_then(|v| v.as_str().parse::().ok()); match (precision, scale) { - (Some(p), Some(s)) => Ok(ArrowDataType::Decimal(p, s)), + // TODO how do we decide which variant (128 / 256) to use? + (Some(p), Some(s)) => Ok(ArrowDataType::Decimal128(p, s)), _ => Err(ArrowError::SchemaError(format!( "Invalid precision or scale decimal type for Arrow: {}", decimal @@ -231,7 +232,11 @@ impl TryFrom<&ArrowDataType> for schema::SchemaDataType { ArrowDataType::Float64 => Ok(schema::SchemaDataType::primitive("double".to_string())), ArrowDataType::Boolean => Ok(schema::SchemaDataType::primitive("boolean".to_string())), ArrowDataType::Binary => Ok(schema::SchemaDataType::primitive("binary".to_string())), - ArrowDataType::Decimal(p, s) => Ok(schema::SchemaDataType::primitive(format!( + ArrowDataType::Decimal128(p, s) => Ok(schema::SchemaDataType::primitive(format!( + "decimal({},{})", + p, s + ))), + ArrowDataType::Decimal256(p, s) => Ok(schema::SchemaDataType::primitive(format!( "decimal({},{})", p, s ))), @@ -690,4 +695,33 @@ mod tests { assert!(expected_fields.contains(&f.name().as_str())); } } + + #[test] + fn test_arrow_from_delta_decimal_type() { + let precision = 20; + let scale = 2; + let decimal_type = String::from(format!["decimal({p},{s})", p = precision, s = scale]); + let decimal_field = crate::SchemaDataType::primitive(decimal_type); + assert_eq!( + >::try_from(&decimal_field).unwrap(), + ArrowDataType::Decimal128(precision, scale) + ); + } + + #[test] + fn test_arrow_from_delta_wrong_decimal_type() { + let precision = 20; + let scale = "wrong"; + let decimal_type = String::from(format!["decimal({p},{s})", p = precision, s = scale]); + let _error = format!( + "Invalid precision or scale decimal type for Arrow: {}", + scale + ); + let decimal_field = crate::SchemaDataType::primitive(decimal_type); + assert!(matches!( + >::try_from(&decimal_field) + .unwrap_err(), + arrow::error::ArrowError::SchemaError(_error), + )); + } } diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 2d5d58a31f..355da573ed 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -26,6 +26,7 @@ use std::convert::TryFrom; use std::sync::Arc; use arrow::array::ArrayRef; +use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, TimeUnit}; use async_trait::async_trait; use chrono::{DateTime, NaiveDateTime, Utc}; @@ -235,12 +236,7 @@ impl PruningStatistics for delta::DeltaTable { statistics .min_values .get(&column.name) - .and_then(|f| { - correct_scalar_value_type( - to_scalar_value(f.as_value()?).unwrap_or(ScalarValue::Null), - &data_type, - ) - }) + .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) .unwrap_or(ScalarValue::Null) } else { ScalarValue::Null @@ -262,12 +258,7 @@ impl PruningStatistics for delta::DeltaTable { statistics .max_values .get(&column.name) - .and_then(|f| { - correct_scalar_value_type( - to_scalar_value(f.as_value()?).unwrap_or(ScalarValue::Null), - &data_type, - ) - }) + .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) .unwrap_or(ScalarValue::Null) } else { ScalarValue::Null @@ -404,17 +395,16 @@ fn partitioned_file_from_action(action: &action::Add, schema: &ArrowSchema) -> P .iter() .filter_map(|f| { action.partition_values.get(f.name()).map(|val| match val { - Some(value) => { - match to_scalar_value(&serde_json::Value::String(value.to_string())) { - Some(parsed) => correct_scalar_value_type(parsed, f.data_type()) - .unwrap_or(ScalarValue::Null), - None => ScalarValue::Null, - } - } + Some(value) => to_correct_scalar_value( + &serde_json::Value::String(value.to_string()), + f.data_type(), + ) + .unwrap_or(ScalarValue::Null), None => ScalarValue::Null, }) }) .collect::>(); + let ts_secs = action.modification_time / 1000; let ts_ns = (action.modification_time % 1000) * 1_000_000; let last_modified = @@ -427,6 +417,7 @@ fn partitioned_file_from_action(action: &action::Add, schema: &ArrowSchema) -> P }, partition_values, range: None, + extensions: None, } } @@ -449,6 +440,51 @@ fn to_scalar_value(stat_val: &serde_json::Value) -> Option Option { + match stat_val { + serde_json::Value::Array(_) => None, + serde_json::Value::Object(_) => None, + serde_json::Value::Null => None, + serde_json::Value::String(string_val) => match field_dt { + ArrowDataType::Timestamp(_, _) => { + let time_nanos = ScalarValue::try_from_string( + string_val.to_owned(), + &ArrowDataType::Timestamp(TimeUnit::Nanosecond, None), + ) + .ok()?; + let cast_arr = cast_with_options( + &time_nanos.to_array(), + field_dt, + &CastOptions { safe: false }, + ) + .ok()?; + Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) + } + _ => Some(ScalarValue::try_from_string(string_val.to_owned(), field_dt).ok()?), + }, + other => match field_dt { + ArrowDataType::Timestamp(_, _) => { + let time_nanos = ScalarValue::try_from_string( + other.to_string(), + &ArrowDataType::Timestamp(TimeUnit::Nanosecond, None), + ) + .ok()?; + let cast_arr = cast_with_options( + &time_nanos.to_array(), + field_dt, + &CastOptions { safe: false }, + ) + .ok()?; + Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) + } + _ => Some(ScalarValue::try_from_string(other.to_string(), field_dt).ok()?), + }, + } +} + fn correct_scalar_value_type( value: datafusion::scalar::ScalarValue, field_dt: &ArrowDataType, @@ -490,7 +526,11 @@ fn correct_scalar_value_type( let raw_value = bool::try_from(value).ok()?; Some(ScalarValue::from(raw_value)) } - ArrowDataType::Decimal(_, _) => { + ArrowDataType::Decimal128(_, _) => { + let raw_value = f64::try_from(value).ok()?; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Decimal256(_, _) => { let raw_value = f64::try_from(value).ok()?; Some(ScalarValue::from(raw_value)) } @@ -572,3 +612,120 @@ fn left_larger_than_right( } } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow::datatypes::Field; + use chrono::{TimeZone, Utc}; + use serde_json::json; + + // test deserialization of serialized partition values. + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#partition-value-serialization + #[test] + fn test_parse_scalar_value() { + let reference_pairs = &[ + ( + json!("2015"), + ArrowDataType::Int16, + ScalarValue::Int16(Some(2015)), + ), + ( + json!("2015"), + ArrowDataType::Int32, + ScalarValue::Int32(Some(2015)), + ), + ( + json!("2015"), + ArrowDataType::Int64, + ScalarValue::Int64(Some(2015)), + ), + ( + json!("2015"), + ArrowDataType::Float32, + ScalarValue::Float32(Some(2015_f32)), + ), + ( + json!("2015"), + ArrowDataType::Float64, + ScalarValue::Float64(Some(2015_f64)), + ), + ( + json!(2015), + ArrowDataType::Float64, + ScalarValue::Float64(Some(2015_f64)), + ), + ( + json!("2015-01-01"), + ArrowDataType::Date32, + ScalarValue::Date32(Some(16436)), + ), + // ( + // json!("2015-01-01"), + // ArrowDataType::Date64, + // ScalarValue::Date64(Some(16436)), + // ), + // TODO(roeap) there seem to be differences in how precisions are handled locally and in CI, need to investigate + // ( + // json!("2020-09-08 13:42:29"), + // ArrowDataType::Timestamp(TimeUnit::Nanosecond, None), + // ScalarValue::TimestampNanosecond(Some(1599565349000000000), None), + // ), + // ( + // json!("2020-09-08 13:42:29"), + // ArrowDataType::Timestamp(TimeUnit::Microsecond, None), + // ScalarValue::TimestampMicrosecond(Some(1599565349000000), None), + // ), + // ( + // json!("2020-09-08 13:42:29"), + // ArrowDataType::Timestamp(TimeUnit::Millisecond, None), + // ScalarValue::TimestampMillisecond(Some(1599565349000), None), + // ), + ( + json!(true), + ArrowDataType::Boolean, + ScalarValue::Boolean(Some(true)), + ), + ]; + + for (raw, data_type, ref_scalar) in reference_pairs { + let scalar = to_correct_scalar_value(raw, data_type).unwrap(); + assert_eq!(*ref_scalar, scalar) + } + } + + #[test] + fn test_partitioned_file_from_action() { + 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 { + path: "year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string(), + size: 10644, + partition_values, + modification_time: 1660497727833, + partition_values_parsed: None, + data_change: true, + stats: None, + stats_parsed: None, + tags: None, + }; + let schema = ArrowSchema::new(vec![ + Field::new("year", ArrowDataType::Int64, true), + Field::new("month", ArrowDataType::Int64, true), + ]); + + let file = partitioned_file_from_action(&action, &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()), + last_modified: Utc.timestamp_millis(1660497727833), + size: 10644, + }, + partition_values: [ScalarValue::Int64(Some(2015)), ScalarValue::Int64(Some(1))].to_vec(), + range: None, + extensions: None, + }; + assert_eq!(file.partition_values, ref_file.partition_values) + } +} diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 55660e8e9e..271bb33f90 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -76,42 +76,16 @@ #![deny(warnings)] #![deny(missing_docs)] -extern crate log; - #[cfg(all(feature = "parquet", feature = "parquet2"))] compile_error!( "Feature parquet and parquet2 are mutually exclusive and cannot be enabled together" ); -#[cfg(feature = "arrow")] -pub use arrow; -#[cfg(feature = "arrow")] -pub mod delta_arrow; - -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod writer; - -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod checkpoints; - -#[cfg(feature = "parquet2")] -pub use parquet2; - -extern crate chrono; -extern crate lazy_static; -extern crate regex; -extern crate serde; -extern crate thiserror; - pub mod action; +pub mod builder; pub mod data_catalog; pub mod delta; pub mod delta_config; -pub mod object_store; -#[cfg(feature = "datafusion-ext")] -pub mod operations; -#[cfg(feature = "parquet")] -pub mod optimize; pub mod partitions; pub mod schema; pub mod storage; @@ -119,20 +93,39 @@ pub mod table_state; pub mod time_utils; pub mod vacuum; -#[cfg(feature = "datafusion-ext")] -pub mod delta_datafusion; - +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod checkpoints; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod delta_arrow; #[cfg(feature = "rust-dataframe-ext")] mod delta_dataframe; +#[cfg(feature = "datafusion-ext")] +pub mod delta_datafusion; +#[cfg(feature = "datafusion-ext")] +pub mod operations; +#[cfg(feature = "parquet")] +pub mod optimize; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod writer; +pub use self::builder::*; pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; pub use self::delta::*; pub use self::partitions::*; pub use self::schema::*; -pub use self::storage::{ - get_backend_for_uri, get_backend_for_uri_with_options, parse_uri, StorageBackend, StorageError, - Uri, UriError, -}; +pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; + +// convenience exports for consumers to avoid aligning crate versions +#[cfg(feature = "arrow")] +pub use arrow; +#[cfg(feature = "datafusion-ext")] +pub use datafusion; +#[cfg(feature = "parquet")] +pub use parquet; +#[cfg(feature = "parquet2")] +pub use parquet2; -#[cfg(any(feature = "s3", feature = "s3-rustls"))] -pub use self::storage::s3::s3_storage_options; +// needed only for integration tests +// TODO can / should we move this into the test crate? +#[cfg(feature = "integration_test")] +pub mod test_utils; diff --git a/rust/src/object_store.rs b/rust/src/object_store.rs deleted file mode 100644 index a7808ecc8f..0000000000 --- a/rust/src/object_store.rs +++ /dev/null @@ -1,508 +0,0 @@ -//! Object Store implementation for DeltaTable. -//! -//! The object store abstracts all interactions with the underlying storage system. -//! Currently local filesystem, S3, Azure, and GCS are supported. -use crate::{ - get_backend_for_uri_with_options, - storage::{ObjectMeta as StorageObjectMeta, StorageBackend, StorageError}, -}; -use bytes::Bytes; -#[cfg(feature = "datafusion-ext")] -use datafusion::datasource::object_store::ObjectStoreUrl; -use futures::stream::BoxStream; -use futures::StreamExt; -use lazy_static::lazy_static; -use object_store::{ - path::{Path, DELIMITER}, - Error as ObjectStoreError, GetResult, ListResult, ObjectMeta, ObjectStore, - Result as ObjectStoreResult, -}; -use std::collections::HashMap; -use std::ops::Range; -use std::sync::Arc; -use url::{ParseError, Url}; - -lazy_static! { - static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); -} - -impl From for ObjectStoreError { - fn from(error: StorageError) -> Self { - match error { - StorageError::NotFound => ObjectStoreError::NotFound { - path: "".to_string(), - source: Box::new(error), - }, - StorageError::AlreadyExists(ref path) => ObjectStoreError::AlreadyExists { - path: path.clone(), - source: Box::new(error), - }, - other => ObjectStoreError::Generic { - store: "DeltaObjectStore", - source: Box::new(other), - }, - } - } -} - -/// Object Store implementation for DeltaTable. -/// -/// The [DeltaObjectStore] implements the [object_store::ObjectStore] trait to facilitate -/// interoperability with the larger rust / arrow ecosystem. Specifically it can directly -/// be registered as store within datafusion. -/// -/// The table root is treated as the root of the object store. -/// All [Path] are reported relative to the table root. -#[derive(Debug, Clone)] -pub struct DeltaObjectStore { - scheme: String, - root: Path, - storage: Arc, -} - -impl std::fmt::Display for DeltaObjectStore { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "DeltaObjectStore({}://{})", self.scheme, self.root) - } -} - -impl DeltaObjectStore { - /// Try creating a new instance of DeltaObjectStore from table uri and storage options - pub fn try_new_with_options( - table_uri: impl AsRef, - storage_options: Option>, - ) -> ObjectStoreResult { - let storage = get_backend_for_uri_with_options( - table_uri.as_ref(), - storage_options.unwrap_or_default(), - ) - .map_err(|err| ObjectStoreError::Generic { - store: "DeltaObjectStore", - source: Box::new(err), - })?; - Self::try_new(table_uri, storage) - } - - /// Try creating a new instance of DeltaObjectStore with specified storage - pub fn try_new( - table_uri: impl AsRef, - storage: Arc, - ) -> ObjectStoreResult { - let (scheme, root) = match Url::parse(table_uri.as_ref()) { - Ok(result) => { - match result.scheme() { - "file" | "gs" | "s3" | "adls2" | "" => { - let raw_path = - format!("{}{}", result.domain().unwrap_or_default(), result.path()); - let root = Path::parse(raw_path)?; - Ok((result.scheme().to_string(), root)) - } - _ => { - // Since we did find some base / scheme, but don't recognize it, it - // may be a local path (i.e. c:/.. on windows). We need to pipe it through path though - // to get consistent path separators. - let local_path = std::path::Path::new(table_uri.as_ref()); - let root = Path::from_filesystem_path(local_path)?; - Ok(("file".to_string(), root)) - } - } - } - Err(ParseError::RelativeUrlWithoutBase) => { - let local_path = std::path::Path::new(table_uri.as_ref()); - let root = Path::from_filesystem_path(local_path)?; - Ok(("file".to_string(), root)) - } - Err(err) => Err(ObjectStoreError::Generic { - store: "DeltaObjectStore", - source: Box::new(err), - }), - }?; - Ok(Self { - scheme, - root, - storage, - }) - } - - /// Get a reference to the underlying storage backend - // TODO we should eventually be able to remove this - pub fn storage_backend(&self) -> Arc { - self.storage.clone() - } - - /// Get fully qualified uri for table root - pub fn root_uri(&self) -> String { - self.to_uri(&Path::from("")) - } - - /// convert a table [Path] to a fully qualified uri - pub fn to_uri(&self, location: &Path) -> String { - let uri = match self.scheme.as_ref() { - "file" | "" => { - // On windows the drive (e.g. 'c:') is part of root and must not be prefixed. - #[cfg(windows)] - let os_uri = format!("{}/{}", self.root, location.as_ref()); - #[cfg(unix)] - let os_uri = format!("/{}/{}", self.root, location.as_ref()); - os_uri - } - _ => format!("{}://{}/{}", self.scheme, self.root, location.as_ref()), - }; - uri.trim_end_matches('/').to_string() - } - - #[cfg(feature = "datafusion-ext")] - /// generate a unique enough url to identify the store in datafusion. - pub(crate) fn object_store_url(&self) -> ObjectStoreUrl { - // we are certain, that the URL can be parsed, since - // we make sure when we are parsing the table uri - ObjectStoreUrl::parse(format!( - "delta-rs://{}", - // NOTE We need to also replace colons, but its fine, since it just needs - // to be a unique-ish identifier for the object store in datafusion - self.root.as_ref().replace(DELIMITER, "-").replace(':', "-") - )) - .expect("Invalid object store url.") - } - - /// [Path] to Delta log - pub fn log_path(&self) -> &Path { - &DELTA_LOG_PATH - } - - /// Deletes object by `paths`. - pub async fn delete_batch(&self, paths: &[Path]) -> ObjectStoreResult<()> { - for path in paths { - match self.delete(path).await { - Ok(_) => continue, - Err(ObjectStoreError::NotFound { .. }) => continue, - Err(e) => return Err(e), - } - } - Ok(()) - } -} - -#[async_trait::async_trait] -impl ObjectStore for DeltaObjectStore { - /// Save the provided bytes to the specified location. - async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult<()> { - Ok(self - .storage - .put_obj(&self.to_uri(location), bytes.as_ref()) - .await?) - } - - /// Return the bytes that are stored at the specified location. - async fn get(&self, location: &Path) -> ObjectStoreResult { - let data = self.storage.get_obj(&self.to_uri(location)).await?; - Ok(GetResult::Stream( - futures::stream::once(async move { Ok(data.into()) }).boxed(), - )) - } - - /// Return the bytes that are stored at the specified location - /// in the given byte range - async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { - let data = self - .storage - .get_range(&self.to_uri(location), range) - .await?; - Ok(data.into()) - } - - /// Return the metadata for the specified location - async fn head(&self, location: &Path) -> ObjectStoreResult { - let meta = self.storage.head_obj(&self.to_uri(location)).await?; - convert_object_meta(self.root_uri(), meta) - } - - /// Delete the object at the specified location. - async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { - Ok(self.storage.delete_obj(&self.to_uri(location)).await?) - } - - /// List all the objects with the given prefix. - /// - /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of - /// `foo/bar_baz/x`. - async fn list( - &self, - prefix: Option<&Path>, - ) -> ObjectStoreResult>> { - let path = match prefix { - Some(pre) => self.to_uri(pre), - None => self.root_uri(), - }; - let root_uri = self.root_uri(); - let stream = self - .storage - .list_objs(&path) - .await? - .map(|obj| match obj { - Ok(meta) => convert_object_meta(root_uri.clone(), meta), - Err(err) => Err(ObjectStoreError::from(err)), - }) - .collect::>() - .await; - Ok(Box::pin(futures::stream::iter(stream))) - } - - /// List objects with the given prefix and an implementation specific - /// delimiter. Returns common prefixes (directories) in addition to object - /// metadata. - /// - /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of - /// `foo/bar_baz/x`. - async fn list_with_delimiter(&self, _prefix: Option<&Path>) -> ObjectStoreResult { - todo!() - } - - /// Copy an object from one path to another in the same object store. - /// - /// If there exists an object at the destination, it will be overwritten. - async fn copy(&self, _from: &Path, _to: &Path) -> ObjectStoreResult<()> { - todo!() - } - - /// Copy an object from one path to another, only if destination is empty. - /// - /// Will return an error if the destination already has an object. - async fn copy_if_not_exists(&self, _from: &Path, _to: &Path) -> ObjectStoreResult<()> { - todo!() - } - - /// Move an object from one path to another in the same object store. - /// - /// Will return an error if the destination already has an object. - async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { - Ok(self - .storage - .rename_obj_noreplace(&self.to_uri(from), &self.to_uri(to)) - .await?) - } -} - -#[inline] -/// Return path relative to parent_path -fn extract_rel_path<'a, 'b>( - parent_path: &'b str, - path: &'a str, -) -> Result<&'a str, ObjectStoreError> { - if path.starts_with(&parent_path) { - Ok(&path[parent_path.len()..]) - } else { - Err(ObjectStoreError::Generic { - store: "DeltaObjectStore", - source: Box::new(StorageError::NotFound), - }) - } -} - -fn convert_object_meta( - root_uri: String, - storage_meta: StorageObjectMeta, -) -> ObjectStoreResult { - Ok(ObjectMeta { - location: Path::from(extract_rel_path( - root_uri.as_ref(), - // HACK hopefully this will hold over until we have switched to object_store - storage_meta.path.as_str().replace('\\', DELIMITER).as_ref(), - )?), - last_modified: storage_meta.modified, - size: storage_meta.size.unwrap_or_default() as usize, - }) -} - -#[cfg(test)] -mod tests { - use super::*; - use futures::TryStreamExt; - use tokio::fs; - - #[tokio::test] - async fn test_put() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - // put object - let tmp_file_path1 = tmp_dir.path().join("tmp_file1"); - let path1 = Path::from("tmp_file1"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - assert!(fs::metadata(tmp_file_path1).await.is_ok()); - - let tmp_file_path2 = tmp_dir.path().join("tmp_dir1").join("file"); - let path2 = Path::from("tmp_dir1/file"); - object_store.put(&path2, bytes::Bytes::new()).await.unwrap(); - assert!(fs::metadata(tmp_file_path2).await.is_ok()) - } - - #[tokio::test] - async fn test_head() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - // existing file - let path1 = Path::from("tmp_file1"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - let meta = object_store.head(&path1).await; - assert!(meta.is_ok()); - - // nonexistent file - let path2 = Path::from("nonexistent"); - let meta = object_store.head(&path2).await; - assert!(meta.is_err()); - } - - #[tokio::test] - async fn test_get() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - // existing file - let path1 = Path::from("tmp_file1"); - let data = bytes::Bytes::from("random data"); - object_store.put(&path1, data.clone()).await.unwrap(); - let data_get = object_store - .get(&path1) - .await - .unwrap() - .bytes() - .await - .unwrap(); - assert_eq!(data, data_get); - } - - #[tokio::test] - async fn test_delete() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - let tmp_file_path1 = tmp_dir.path().join("tmp_file1"); - - // put object - let path1 = Path::from("tmp_file1"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - assert!(fs::metadata(tmp_file_path1.clone()).await.is_ok()); - - // delete object - object_store.delete(&path1).await.unwrap(); - assert!(fs::metadata(tmp_file_path1).await.is_err()); - } - - #[tokio::test] - async fn test_delete_batch() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - let tmp_file_path1 = tmp_dir.path().join("tmp_file1"); - let tmp_file_path2 = tmp_dir.path().join("tmp_file2"); - - // put object - let path1 = Path::from("tmp_file1"); - let path2 = Path::from("tmp_file2"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - object_store.put(&path2, bytes::Bytes::new()).await.unwrap(); - assert!(fs::metadata(tmp_file_path1.clone()).await.is_ok()); - assert!(fs::metadata(tmp_file_path2.clone()).await.is_ok()); - - // delete objects - object_store.delete_batch(&[path1, path2]).await.unwrap(); - assert!(fs::metadata(tmp_file_path1).await.is_err()); - assert!(fs::metadata(tmp_file_path2).await.is_err()) - } - - #[tokio::test] - async fn test_list() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - let path1 = Path::from("tmp_file1"); - let path2 = Path::from("tmp_file2"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - object_store.put(&path2, bytes::Bytes::new()).await.unwrap(); - - let objs = object_store - .list(None) - .await - .unwrap() - .try_collect::>() - .await - .unwrap(); - assert_eq!(objs.len(), 2); - - let path1 = Path::from("prefix/tmp_file1"); - let path2 = Path::from("prefix/tmp_file2"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - object_store.put(&path2, bytes::Bytes::new()).await.unwrap(); - - let objs = object_store - .list(None) - .await - .unwrap() - .try_collect::>() - .await - .unwrap(); - assert_eq!(objs.len(), 4); - - let objs = object_store - .list(Some(&Path::from("prefix"))) - .await - .unwrap() - .try_collect::>() - .await - .unwrap(); - assert_eq!(objs.len(), 2) - } - - #[tokio::test] - async fn test_list_prefix() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - let path1 = Path::from("_delta_log/tmp_file1"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - - let objs = object_store - .list(None) - .await - .unwrap() - .try_collect::>() - .await - .unwrap(); - assert_eq!(objs[0].location, path1) - } - - #[tokio::test] - async fn test_rename_if_not_exists() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); - let object_store = - DeltaObjectStore::try_new_with_options(tmp_dir.path().to_str().unwrap(), None).unwrap(); - - let tmp_file_path1 = tmp_dir.path().join("tmp_file1"); - let tmp_file_path2 = tmp_dir.path().join("tmp_file2"); - - let path1 = Path::from("tmp_file1"); - let path2 = Path::from("tmp_file2"); - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - - // delete objects - let result = object_store.rename_if_not_exists(&path1, &path2).await; - assert!(result.is_ok()); - assert!(fs::metadata(tmp_file_path1.clone()).await.is_err()); - assert!(fs::metadata(tmp_file_path2.clone()).await.is_ok()); - - object_store.put(&path1, bytes::Bytes::new()).await.unwrap(); - let result = object_store.rename_if_not_exists(&path1, &path2).await; - assert!(result.is_err()); - assert!(fs::metadata(tmp_file_path1).await.is_ok()); - assert!(fs::metadata(tmp_file_path2).await.is_ok()); - } -} diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 7a1d42a92c..ca5e3e6a54 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -1,13 +1,13 @@ //! Command for creating a new delta table // https://github.com/delta-io/delta/blob/master/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala use super::{ - get_table_from_uri_without_update, to_datafusion_err, + to_datafusion_err, transaction::{serialize_actions, OPERATION_SCHEMA}, DeltaCommandError, *, }; use crate::{ action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}, - DeltaTableMetaData, + DeltaTableBuilder, DeltaTableMetaData, }; use async_trait::async_trait; use core::any::Any; @@ -136,8 +136,9 @@ async fn do_create( metadata: DeltaTableMetaData, protocol: Protocol, ) -> DataFusionResult { - let mut table = - get_table_from_uri_without_update(table_uri.clone()).map_err(to_datafusion_err)?; + let mut table = DeltaTableBuilder::from_uri(&table_uri) + .build() + .map_err(to_datafusion_err)?; let actions = match table.load_version(0).await { Err(_) => Ok(vec![ diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 8aaafba710..84aec67f11 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -3,11 +3,11 @@ // - rename to delta operations use crate::{ action::{DeltaOperation, Protocol, SaveMode}, - get_backend_for_uri_with_options, open_table, + builder::DeltaTableBuilder, + open_table, operations::{create::CreateCommand, transaction::DeltaTransactionPlan, write::WriteCommand}, - storage::StorageError, writer::{record_batch::divide_by_partition_values, utils::PartitionPath, DeltaWriterError}, - DeltaTable, DeltaTableConfig, DeltaTableError, DeltaTableMetaData, + DeltaTable, DeltaTableError, DeltaTableMetaData, }; use arrow::{datatypes::SchemaRef as ArrowSchemaRef, error::ArrowError, record_batch::RecordBatch}; use datafusion::{ @@ -46,31 +46,23 @@ pub enum DeltaCommandError { TableAlreadyExists(String), /// Error returned when errors occur in underlying delta table instance - #[error("Error in underlying DeltaTable")] - DeltaTableError { + #[error("DeltaTable error: {} ({:?})", source, source)] + DeltaTable { /// Raw internal DeltaTableError #[from] source: DeltaTableError, }, /// Errors occurring inside the DeltaWriter modules - #[error("Error in underlying DeltaWriter")] - DeltaWriter { + #[error("Writer error: {} ({:?})", source, source)] + Writer { /// Raw internal DeltaWriterError #[from] source: DeltaWriterError, }, - /// Error returned when errors occur in underlying storage instance - #[error("Error in underlying storage backend")] - Storage { - /// Raw internal StorageError - #[from] - source: StorageError, - }, - /// Error returned when errors occur in Arrow - #[error("Error handling arrow data")] + #[error("Arrow error: {} ({:?})", source, source)] Arrow { /// Raw internal ArrowError #[from] @@ -78,12 +70,29 @@ pub enum DeltaCommandError { }, /// Error returned for errors internal to Datafusion - #[error("Error in Datafusion execution engine")] + #[error("Datafusion error: {} ({:?})", source, source)] DataFusion { /// Raw internal DataFusionError - #[from] source: DataFusionError, }, + + /// Error returned for errors internal to Datafusion + #[error("ObjectStore error: {} ({:?})", source, source)] + ObjectStore { + /// Raw internal DataFusionError + #[from] + source: object_store::Error, + }, +} + +impl From for DeltaCommandError { + fn from(err: DataFusionError) -> Self { + match err { + DataFusionError::ArrowError(source) => DeltaCommandError::Arrow { source }, + DataFusionError::ObjectStore(source) => DeltaCommandError::ObjectStore { source }, + source => DeltaCommandError::DataFusion { source }, + } + } } fn to_datafusion_err(e: impl std::error::Error) -> DataFusionError { @@ -105,7 +114,7 @@ impl DeltaCommands { let table = if let Ok(tbl) = open_table(&table_uri).await { Ok(tbl) } else { - get_table_from_uri_without_update(table_uri) + DeltaTableBuilder::from_uri(table_uri).build() }?; Ok(Self { table }) } @@ -121,7 +130,7 @@ impl DeltaCommands { plan: Arc, ) -> DeltaCommandResult<()> { let transaction = Arc::new(DeltaTransactionPlan::new( - self.table.table_uri.clone(), + self.table.table_uri(), self.table.version(), plan, operation, @@ -145,7 +154,7 @@ impl DeltaCommands { let operation = DeltaOperation::Create { mode, metadata: metadata.clone(), - location: self.table.table_uri.clone(), + location: self.table.table_uri(), // TODO get the protocol from somewhere central protocol: Protocol { min_reader_version: 1, @@ -153,7 +162,7 @@ impl DeltaCommands { }, }; let plan = Arc::new(CreateCommand::try_new( - &self.table.table_uri, + self.table.table_uri(), operation.clone(), )?); @@ -216,7 +225,7 @@ impl DeltaCommands { }; let data_plan = Arc::new(MemoryExec::try_new(&data, schema, None)?); let plan = Arc::new(WriteCommand::try_new( - &self.table.table_uri, + self.table.table_uri(), operation.clone(), data_plan, )?); @@ -224,13 +233,6 @@ impl DeltaCommands { } } -fn get_table_from_uri_without_update(table_uri: String) -> DeltaCommandResult { - let backend = get_backend_for_uri_with_options(&table_uri, HashMap::new())?; - let table = DeltaTable::new(&table_uri, backend, DeltaTableConfig::default())?; - - Ok(table) -} - impl From for DeltaCommands { fn from(table: DeltaTable) -> Self { Self { table } @@ -284,7 +286,7 @@ mod tests { let mut table = create_initialized_table(&partition_cols).await; assert_eq!(table.version(), 0); - let mut commands = DeltaCommands::try_from_uri(table.table_uri.to_string()) + let mut commands = DeltaCommands::try_from_uri(table.table_uri()) .await .unwrap(); diff --git a/rust/src/operations/transaction.rs b/rust/src/operations/transaction.rs index 7c312373b2..b1b183eac9 100644 --- a/rust/src/operations/transaction.rs +++ b/rust/src/operations/transaction.rs @@ -148,8 +148,9 @@ async fn do_transaction( app_metadata: Option>, context: Arc, ) -> DataFusionResult { - let mut table = - get_table_from_uri_without_update(table_uri.clone()).map_err(to_datafusion_err)?; + let mut table = DeltaTableBuilder::from_uri(table_uri) + .build() + .map_err(to_datafusion_err)?; let schema = input.schema().clone(); let data = collect(input, context.clone()).await?; diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index da362bef9f..c1355f1528 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -177,8 +177,9 @@ async fn do_write( mode: SaveMode, context: Arc, ) -> DataFusionResult { - let mut table = - get_table_from_uri_without_update(table_uri.clone()).map_err(to_datafusion_err)?; + let mut table = DeltaTableBuilder::from_uri(&table_uri) + .build() + .map_err(to_datafusion_err)?; let metrics = ExecutionPlanMetricsSet::new(); let tracking_metrics = MemTrackingMetrics::new(&metrics, partition_id); @@ -425,7 +426,7 @@ mod tests { let mut table = create_initialized_table(&partition_cols).await; assert_eq!(table.version(), 0); - let transaction = get_transaction(table.table_uri.clone(), 0, SaveMode::Append); + let transaction = get_transaction(table.table_uri(), 0, SaveMode::Append); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); @@ -436,7 +437,7 @@ mod tests { assert_eq!(table.get_file_uris().collect::>().len(), 2); assert_eq!(table.version(), 1); - let transaction = get_transaction(table.table_uri.clone(), 1, SaveMode::Append); + let transaction = get_transaction(table.table_uri(), 1, SaveMode::Append); let _ = collect(transaction.clone(), task_ctx).await.unwrap(); table.update().await.unwrap(); assert_eq!(table.get_file_uris().collect::>().len(), 4); @@ -449,7 +450,7 @@ mod tests { let mut table = create_initialized_table(&partition_cols).await; assert_eq!(table.version(), 0); - let transaction = get_transaction(table.table_uri.clone(), 0, SaveMode::Overwrite); + let transaction = get_transaction(table.table_uri(), 0, SaveMode::Overwrite); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); @@ -460,7 +461,7 @@ mod tests { assert_eq!(table.get_file_uris().collect::>().len(), 2); assert_eq!(table.version(), 1); - let transaction = get_transaction(table.table_uri.clone(), 1, SaveMode::Overwrite); + let transaction = get_transaction(table.table_uri(), 1, SaveMode::Overwrite); let _ = collect(transaction.clone(), task_ctx).await.unwrap(); table.update().await.unwrap(); assert_eq!(table.get_file_uris().collect::>().len(), 2); diff --git a/rust/src/storage/azure/mod.rs b/rust/src/storage/azure/mod.rs deleted file mode 100644 index cc8283860b..0000000000 --- a/rust/src/storage/azure/mod.rs +++ /dev/null @@ -1,468 +0,0 @@ -//! The Azure Data Lake Storage Gen2 storage backend. -//! -//! This module is gated behind the "azure" feature. -//! -use super::{parse_uri, str_option, ObjectMeta, StorageBackend, StorageError, UriError}; -use azure_core::auth::TokenCredential; -use azure_core::{error::ErrorKind as AzureErrorKind, ClientOptions}; -use azure_identity::{ - AutoRefreshingTokenCredential, ClientSecretCredential, TokenCredentialOptions, -}; -use azure_storage::storage_shared_key_credential::StorageSharedKeyCredential; -use azure_storage_datalake::prelude::*; -use futures::stream::{self, BoxStream}; -use futures::{future::Either, StreamExt}; -use log::debug; -use std::collections::HashMap; -use std::fmt; -use std::fmt::Debug; -use std::ops::Range; -use std::sync::Arc; - -/// Storage option keys to use when creating [crate::storage::azure::AzureStorageOptions]. -/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. -pub mod azure_storage_options { - ///The ADLS Gen2 Access Key - pub const AZURE_STORAGE_ACCOUNT_KEY: &str = "AZURE_STORAGE_ACCOUNT_KEY"; - ///The name of storage account - pub const AZURE_STORAGE_ACCOUNT_NAME: &str = "AZURE_STORAGE_ACCOUNT_NAME"; - /// Connection string for connecting to azure storage account - pub const AZURE_STORAGE_CONNECTION_STRING: &str = "AZURE_STORAGE_CONNECTION_STRING"; - /// Service principal id - pub const AZURE_CLIENT_ID: &str = "AZURE_CLIENT_ID"; - /// Service principal secret - pub const AZURE_CLIENT_SECRET: &str = "AZURE_CLIENT_SECRET"; - /// ID for Azure (AAD) tenant where service principal is registered. - pub const AZURE_TENANT_ID: &str = "AZURE_TENANT_ID"; -} - -/// Options used to configure the AdlsGen2Backend. -/// -/// Available options are described in [azure_storage_options]. -#[derive(Clone, Debug, PartialEq, Eq)] -pub struct AzureStorageOptions { - account_key: Option, - account_name: Option, - // connection_string: Option, - client_id: Option, - client_secret: Option, - tenant_id: Option, -} - -impl AzureStorageOptions { - /// Creates an empty instance of AzureStorageOptions - pub fn new() -> Self { - Self { - account_key: None, - account_name: None, - client_id: None, - client_secret: None, - tenant_id: None, - } - } - - /// Creates an instance of AzureStorageOptions from the given HashMap and environment variables. - pub fn from_map(options: HashMap) -> Self { - Self { - account_key: str_option(&options, azure_storage_options::AZURE_STORAGE_ACCOUNT_KEY), - account_name: str_option(&options, azure_storage_options::AZURE_STORAGE_ACCOUNT_NAME), - // connection_string: str_option( - // &options, - // azure_storage_options::AZURE_STORAGE_CONNECTION_STRING, - // ), - client_id: str_option(&options, azure_storage_options::AZURE_CLIENT_ID), - client_secret: str_option(&options, azure_storage_options::AZURE_CLIENT_SECRET), - tenant_id: str_option(&options, azure_storage_options::AZURE_TENANT_ID), - } - } - - /// set account name - pub fn with_account_name(&mut self, account_name: impl Into) -> &mut Self { - self.account_name = Some(account_name.into()); - self - } - - /// set account key - pub fn with_account_key(&mut self, account_key: impl Into) -> &mut Self { - self.account_key = Some(account_key.into()); - self - } - - /// set client id - pub fn with_client_id(&mut self, client_id: impl Into) -> &mut Self { - self.client_id = Some(client_id.into()); - self - } - - /// set client secret - pub fn with_client_secret(&mut self, client_secret: impl Into) -> &mut Self { - self.client_secret = Some(client_secret.into()); - self - } - - /// set tenant id - pub fn with_tenant_id(&mut self, tenant_id: impl Into) -> &mut Self { - self.tenant_id = Some(tenant_id.into()); - self - } -} - -impl Default for AzureStorageOptions { - /// Creates an instance of AzureStorageOptions from environment variables. - fn default() -> AzureStorageOptions { - Self::from_map(HashMap::new()) - } -} - -impl TryInto for AzureStorageOptions { - type Error = StorageError; - - fn try_into(self) -> Result { - let account_name = self.account_name.ok_or_else(|| { - StorageError::AzureConfig("account name must be provided".to_string()) - })?; - - if let Some(account_key) = self.account_key { - let key = StorageSharedKeyCredential::new(account_name, account_key); - return Ok(DataLakeClient::new_with_shared_key( - key, - None, - ClientOptions::default(), - )); - } - - let client_id = self.client_id.ok_or_else(|| { - StorageError::AzureConfig("account key or client config must be provided".to_string()) - })?; - let client_secret = self.client_secret.ok_or_else(|| { - StorageError::AzureConfig("account key or client config must be provided".to_string()) - })?; - let tenant_id = self.tenant_id.ok_or_else(|| { - StorageError::AzureConfig("account key or client config must be provided".to_string()) - })?; - - let client_credential = Arc::new(ClientSecretCredential::new( - tenant_id, - client_id, - client_secret, - TokenCredentialOptions::default(), - )); - - Ok(DataLakeClient::new_with_token_credential( - Arc::new(AutoRefreshingTokenCredential::new(client_credential)), - account_name, - None, - ClientOptions::default(), - )) - } -} - -/// An object on an Azure Data Lake Storage Gen2 account. -#[derive(Debug, PartialEq, Eq)] -pub struct AdlsGen2Object<'a> { - /// The storage account name. - pub account_name: &'a str, - /// The container, or filesystem, of the object. - pub file_system: &'a str, - /// The path of the object on the filesystem. - pub path: &'a str, -} - -impl<'a> fmt::Display for AdlsGen2Object<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - // This URI syntax is an invention of delta-rs. - // ABFS URIs should not be used since delta-rs doesn't use the Hadoop ABFS driver. - write!( - f, - "adls2://{}/{}/{}", - self.account_name, self.file_system, self.path - ) - } -} - -/// A storage backend for use with an Azure Data Lake Storage Gen2 account (HNS=enabled). -/// -/// This uses the `dfs.core.windows.net` endpoint. -#[derive(Debug)] -pub struct AdlsGen2Backend { - file_system_name: String, - file_system_client: FileSystemClient, -} - -impl AdlsGen2Backend { - /// Create a new [`AdlsGen2Backend`]. - /// - /// This will try to parse configuration options from the environment. - /// - /// The variable `AZURE_STORAGE_ACCOUNT_NAME` always has to be set. - /// - /// To use shared key authorization, also set: - /// * `AZURE_STORAGE_ACCOUNT_KEY` - /// - /// To use a service principal, set: - /// * `AZURE_CLIENT_ID` - /// * `AZURE_CLIENT_SECRET` - /// * `AZURE_TENANT_ID` - /// - /// If both are configured in the environment, shared key authorization will take precedence. - /// - /// See `new_with_token_credential` to pass your own [azure_core::auth::TokenCredential] - /// - /// See `new_from_options` for more fine grained control using [AzureStorageOptions] - pub fn new(file_system_name: impl Into + Clone) -> Result { - Self::new_from_options(file_system_name, AzureStorageOptions::default()) - } - - /// Create a new [`AdlsGen2Backend`] using a [`TokenCredential`] - /// See [`azure_core::auth::TokenCredential`] for various implementations - pub fn new_with_token_credential( - storage_account_name: impl Into, - file_system_name: impl Into + Clone, - token_credential: Arc, - ) -> Result { - let storage_account_name: String = storage_account_name.into(); - let data_lake_client = DataLakeClient::new_with_token_credential( - token_credential, - storage_account_name, - None, - ClientOptions::default(), - ); - - let file_system_client = data_lake_client.into_file_system_client(file_system_name.clone()); - - Ok(AdlsGen2Backend { - file_system_name: file_system_name.into(), - file_system_client, - }) - } - - /// Create a new [`AdlsGen2Backend`] using shared key authentication - pub fn new_with_shared_key( - storage_account_name: impl Into, - file_system_name: impl Into + Clone, - storage_account_key: impl Into, - ) -> Result { - let mut options = AzureStorageOptions::new(); - let options = options - .with_account_name(storage_account_name) - .with_account_key(storage_account_key); - - Self::new_from_options(file_system_name, options.clone()) - } - - /// Create a new [`AdlsGen2Backend`] using a service principal - pub fn new_with_client( - storage_account_name: impl Into, - file_system_name: impl Into + Clone, - client_id: impl Into, - client_secret: impl Into, - tenant_id: impl Into, - ) -> Result { - let mut options = AzureStorageOptions::new(); - let options = options - .with_account_name(storage_account_name) - .with_client_id(client_id) - .with_client_secret(client_secret) - .with_tenant_id(tenant_id); - - Self::new_from_options(file_system_name, options.clone()) - } - - /// Create a new [`AdlsGen2Backend`] from AzureStorageOptions - /// - /// see [azure_storage_options] for the available configuration keys. - /// - /// ```rust,ignore - /// let mut options = AzureStorageOptions::new(); - /// - /// let options = options - /// .with_account_name("") - /// .with_account_key(""); - /// - /// let backend = AdlsGen2Backend::new_from_options("", options.clone()); - /// ``` - pub fn new_from_options( - file_system_name: impl Into + Clone, - options: AzureStorageOptions, - ) -> Result { - let data_lake_client: DataLakeClient = options.try_into()?; - let file_system_client = data_lake_client.into_file_system_client(file_system_name.clone()); - - Ok(AdlsGen2Backend { - file_system_name: file_system_name.into(), - file_system_client, - }) - } - - fn validate_container<'a>(&self, obj: &AdlsGen2Object<'a>) -> Result<(), StorageError> { - if obj.file_system != self.file_system_name { - Err(StorageError::Uri { - source: UriError::ContainerMismatch { - expected: self.file_system_name.clone(), - got: obj.file_system.to_string(), - }, - }) - } else { - Ok(()) - } - } -} - -#[async_trait::async_trait] -impl StorageBackend for AdlsGen2Backend { - async fn head_obj(&self, path: &str) -> Result { - debug!("Getting properties for {}", path); - let obj = parse_uri(path)?.into_adlsgen2_object()?; - self.validate_container(&obj)?; - - let properties = self - .file_system_client - .get_file_client(obj.path) - .get_properties() - .into_future() - .await?; - - let modified = properties.last_modified; - Ok(ObjectMeta { - path: path.to_string(), - modified, - size: properties.content_length, - }) - } - - async fn get_obj(&self, path: &str) -> Result, StorageError> { - debug!("Loading {}", path); - let obj = parse_uri(path)?.into_adlsgen2_object()?; - self.validate_container(&obj)?; - - let data = self - .file_system_client - .get_file_client(obj.path) - .read() - .into_future() - .await? - .data - .to_vec(); - Ok(data) - } - - /// Fetch a range from object content - async fn get_range(&self, path: &str, range: Range) -> Result, StorageError> { - let obj = parse_uri(path)?.into_adlsgen2_object()?; - self.validate_container(&obj)?; - - let data = self - .file_system_client - .get_file_client(obj.path) - .read() - .range(range) - .into_future() - .await? - .data - .to_vec(); - Ok(data) - } - - async fn list_objs<'a>( - &'a self, - path: &'a str, - ) -> Result>, StorageError> { - debug!("Listing objects under {}", path); - let obj = parse_uri(path)?.into_adlsgen2_object()?; - self.validate_container(&obj)?; - - Ok(self - .file_system_client - .list_paths() - .directory(obj.path) - .into_stream() - .flat_map(|it| match it { - Ok(paths) => Either::Left(stream::iter(paths.into_iter().filter_map(|p| { - if p.is_directory { - None - } else { - Some(Ok(ObjectMeta { - path: format!( - "adls2://{}/{}/{}", - obj.account_name.to_owned(), - self.file_system_name, - p.name - ), - modified: p.last_modified, - size: Some(p.content_length), - })) - } - }))), - Err(err) => Either::Right(stream::once(async { - Err(StorageError::Azure { source: err }) - })), - }) - .boxed()) - } - - async fn put_obj(&self, path: &str, obj_bytes: &[u8]) -> Result<(), StorageError> { - let obj = parse_uri(path)?.into_adlsgen2_object()?; - self.validate_container(&obj)?; - - let data = bytes::Bytes::from(obj_bytes.to_owned()); // TODO: Review obj_bytes.to_owned() - let length = data.len() as i64; - - // TODO: Consider using Blob API again since it's just 1 REST call instead of 3 - let file_client = self.file_system_client.get_file_client(obj.path); - file_client.create().into_future().await?; - file_client.append(0, data).into_future().await?; - file_client.flush(length).close(true).into_future().await?; - - Ok(()) - } - - async fn rename_obj_noreplace(&self, src: &str, dst: &str) -> Result<(), StorageError> { - let src_obj = parse_uri(src)?.into_adlsgen2_object()?; - self.validate_container(&src_obj)?; - - let dst_obj = parse_uri(dst)?.into_adlsgen2_object()?; - self.validate_container(&dst_obj)?; - - self.file_system_client - .get_file_client(src_obj.path) - .rename_if_not_exists(dst_obj.path) - .into_future() - .await - .map_err(|err| match err.kind() { - AzureErrorKind::HttpResponse { status, .. } if *status == 409 => { - StorageError::AlreadyExists(dst.to_string()) - } - _ => err.into(), - })?; - - Ok(()) - } - - async fn delete_obj(&self, path: &str) -> Result<(), StorageError> { - let obj = parse_uri(path)?.into_adlsgen2_object()?; - self.validate_container(&obj)?; - - let file_client = self.file_system_client.get_file_client(obj.path); - file_client.delete().into_future().await?; - Ok(()) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn parse_azure_object_uri() { - let uri = parse_uri("adls2://my_account_name/my_file_system_name/my_path").unwrap(); - assert_eq!(uri.path(), "my_path"); - assert_eq!( - uri.into_adlsgen2_object().unwrap(), - AdlsGen2Object { - account_name: "my_account_name", - file_system: "my_file_system_name", - path: "my_path", - } - ); - } -} diff --git a/rust/src/storage/file.rs b/rust/src/storage/file.rs new file mode 100644 index 0000000000..548475c7fd --- /dev/null +++ b/rust/src/storage/file.rs @@ -0,0 +1,380 @@ +//! Local file storage backend. This backend read and write objects from local filesystem. +//! +//! The local file storage backend is multi-writer safe. + +use bytes::Bytes; +use futures::stream::BoxStream; +use object_store::{ + local::LocalFileSystem, path::Path as ObjectStorePath, Error as ObjectStoreError, GetResult, + ListResult, MultipartId, ObjectMeta as ObjStoreObjectMeta, ObjectStore, + Result as ObjectStoreResult, +}; +use std::ops::Range; +use std::sync::Arc; +use tokio::io::AsyncWrite; + +const STORE_NAME: &str = "DeltaLocalObjectStore"; + +/// Error raised by storage lock client +#[derive(thiserror::Error, Debug)] +#[allow(dead_code)] +pub(self) enum LocalFileSystemError { + #[error("Object exists already at path: {} ({:?})", path, source)] + AlreadyExists { + path: String, + source: Box, + }, + + #[error("Object not found at path: {} ({:?})", path, source)] + NotFound { + path: String, + source: Box, + }, + + #[error("Invalid argument in OS call for path: {} ({:?})", path, source)] + InvalidArgument { path: String, source: errno::Errno }, + + #[error("Null error in FFI for path: {} ({:?})", path, source)] + NullError { + path: String, + source: std::ffi::NulError, + }, + + #[error("Generic error in store: {} ({:?})", store, source)] + Generic { + store: &'static str, + source: Box, + }, + + #[error("Error executing async task for path: {} ({:?})", path, source)] + Tokio { + path: String, + source: tokio::task::JoinError, + }, +} + +impl From for ObjectStoreError { + fn from(e: LocalFileSystemError) -> Self { + match e { + LocalFileSystemError::AlreadyExists { path, source } => { + ObjectStoreError::AlreadyExists { path, source } + } + LocalFileSystemError::NotFound { path, source } => { + ObjectStoreError::NotFound { path, source } + } + LocalFileSystemError::InvalidArgument { source, .. } => ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(source), + }, + LocalFileSystemError::NullError { source, .. } => ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(source), + }, + LocalFileSystemError::Tokio { source, .. } => ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(source), + }, + LocalFileSystemError::Generic { store, source } => { + ObjectStoreError::Generic { store, source } + } + } + } +} + +/// Multi-writer support for different platforms: +/// +/// * Modern Linux kernels are well supported. However because Linux implementation leverages +/// `RENAME_NOREPLACE`, older versions of the kernel might not work depending on what filesystem is +/// being used: +/// * ext4 requires >= Linux 3.15 +/// * btrfs, shmem, and cif requires >= Linux 3.17 +/// * xfs requires >= Linux 4.0 +/// * ext2, minix, reiserfs, jfs, vfat, and bpf requires >= Linux 4.9 +/// * Darwin is supported but not fully tested. +/// Patches welcome. +/// * Support for other platforms are not implemented at the moment. +#[derive(Debug, Default)] +pub struct FileStorageBackend { + inner: Arc, +} + +impl FileStorageBackend { + /// Creates a new FileStorageBackend. + pub fn new() -> Self { + Self { + inner: Arc::new(LocalFileSystem::default()), + } + } +} + +impl std::fmt::Display for FileStorageBackend { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "FileStorageBackend") + } +} + +/// Return an absolute filesystem path of the given location +fn path_to_filesystem(location: &ObjectStorePath) -> String { + let mut url = url::Url::parse("file:///").unwrap(); + url.path_segments_mut() + .expect("url path") + // technically not necessary as Path ignores empty segments + // but avoids creating paths with "//" which look odd in error messages. + .pop_if_empty() + .extend(location.parts()); + + url.to_file_path().unwrap().to_str().unwrap().to_owned() +} + +#[async_trait::async_trait] +impl ObjectStore for FileStorageBackend { + async fn put(&self, location: &ObjectStorePath, bytes: Bytes) -> ObjectStoreResult<()> { + self.inner.put(location, bytes).await + } + + async fn get(&self, location: &ObjectStorePath) -> ObjectStoreResult { + self.inner.get(location).await + } + + async fn get_range( + &self, + location: &ObjectStorePath, + range: Range, + ) -> ObjectStoreResult { + self.inner.get_range(location, range).await + } + + async fn head(&self, location: &ObjectStorePath) -> ObjectStoreResult { + self.inner.head(location).await + } + + async fn delete(&self, location: &ObjectStorePath) -> ObjectStoreResult<()> { + self.inner.delete(location).await + } + + async fn list( + &self, + prefix: Option<&ObjectStorePath>, + ) -> ObjectStoreResult>> { + self.inner.list(prefix).await + } + + async fn list_with_delimiter( + &self, + prefix: Option<&ObjectStorePath>, + ) -> ObjectStoreResult { + self.inner.list_with_delimiter(prefix).await + } + + async fn copy(&self, from: &ObjectStorePath, to: &ObjectStorePath) -> ObjectStoreResult<()> { + self.inner.copy(from, to).await + } + + async fn copy_if_not_exists( + &self, + from: &ObjectStorePath, + to: &ObjectStorePath, + ) -> ObjectStoreResult<()> { + self.inner.copy_if_not_exists(from, to).await + } + + async fn rename_if_not_exists( + &self, + from: &ObjectStorePath, + to: &ObjectStorePath, + ) -> ObjectStoreResult<()> { + let path_from = path_to_filesystem(from); + let path_to = path_to_filesystem(to); + Ok(rename_noreplace(path_from.as_ref(), path_to.as_ref()).await?) + } + + async fn put_multipart( + &self, + location: &ObjectStorePath, + ) -> ObjectStoreResult<(MultipartId, Box)> { + self.inner.put_multipart(location).await + } + + async fn abort_multipart( + &self, + location: &ObjectStorePath, + multipart_id: &MultipartId, + ) -> ObjectStoreResult<()> { + self.inner.abort_multipart(location, multipart_id).await + } +} + +/// Atomically renames `from` to `to`. +/// `from` has to exist, but `to` is not, otherwise the operation will fail. +#[inline] +async fn rename_noreplace(from: &str, to: &str) -> Result<(), LocalFileSystemError> { + imp::rename_noreplace(from, to).await +} + +// Generic implementation (Requires 2 system calls) +#[cfg(not(any( + all(target_os = "linux", target_env = "gnu", glibc_renameat2), + target_os = "macos" +)))] +mod imp { + use super::*; + + pub(super) async fn rename_noreplace(from: &str, to: &str) -> Result<(), LocalFileSystemError> { + let from_path = String::from(from); + let to_path = String::from(to); + + tokio::task::spawn_blocking(move || { + std::fs::hard_link(&from_path, &to_path).map_err(|err| { + if err.kind() == std::io::ErrorKind::AlreadyExists { + LocalFileSystemError::AlreadyExists { + path: to_path, + source: Box::new(err), + } + } else if err.kind() == std::io::ErrorKind::NotFound { + LocalFileSystemError::NotFound { + path: from_path.clone(), + source: Box::new(err), + } + } else { + LocalFileSystemError::Generic { + store: STORE_NAME, + source: Box::new(err), + } + } + })?; + + std::fs::remove_file(from_path).map_err(|err| LocalFileSystemError::Generic { + store: STORE_NAME, + source: Box::new(err), + })?; + + Ok(()) + }) + .await + .unwrap() + } +} + +// Optimized implementations (Only 1 system call) +#[cfg(any( + all(target_os = "linux", target_env = "gnu", glibc_renameat2), + target_os = "macos" +))] +mod imp { + use super::*; + use std::ffi::CString; + + fn to_c_string(p: &str) -> Result { + CString::new(p).map_err(|err| LocalFileSystemError::NullError { + path: p.into(), + source: err, + }) + } + + pub(super) async fn rename_noreplace(from: &str, to: &str) -> Result<(), LocalFileSystemError> { + let cs_from = to_c_string(from)?; + let cs_to = to_c_string(to)?; + + let ret = unsafe { + tokio::task::spawn_blocking(move || { + let ret = platform_specific_rename(cs_from.as_ptr(), cs_to.as_ptr()); + if ret != 0 { + Err(errno::errno()) + } else { + Ok(()) + } + }) + .await + .map_err(|err| LocalFileSystemError::Tokio { + path: from.into(), + source: err, + })? + }; + + match ret { + Err(e) if e.0 == libc::EEXIST => Err(LocalFileSystemError::AlreadyExists { + path: to.into(), + source: Box::new(e), + }), + Err(e) if e.0 == libc::ENOENT => Err(LocalFileSystemError::NotFound { + path: to.into(), + source: Box::new(e), + }), + Err(e) if e.0 == libc::EINVAL => Err(LocalFileSystemError::InvalidArgument { + path: to.into(), + source: e, + }), + Err(e) => Err(LocalFileSystemError::Generic { + store: STORE_NAME, + source: Box::new(e), + }), + Ok(_) => Ok(()), + } + } + + #[allow(unused_variables)] + unsafe fn platform_specific_rename(from: *const libc::c_char, to: *const libc::c_char) -> i32 { + cfg_if::cfg_if! { + if #[cfg(all(target_os = "linux", target_env = "gnu"))] { + libc::renameat2(libc::AT_FDCWD, from, libc::AT_FDCWD, to, libc::RENAME_NOREPLACE) + } else if #[cfg(target_os = "macos")] { + libc::renamex_np(from, to, libc::RENAME_EXCL) + } else { + unreachable!() + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::fs::File; + use std::io::Write; + use std::path::{Path, PathBuf}; + + #[tokio::test()] + async fn test_rename_noreplace() { + let tmp_dir = tempdir::TempDir::new_in(".", "test_rename_noreplace").unwrap(); + let a = create_file(&tmp_dir.path(), "a"); + let b = create_file(&tmp_dir.path(), "b"); + let c = &tmp_dir.path().join("c"); + + // unsuccessful move not_exists to C, not_exists is missing + let result = rename_noreplace("not_exists", c.to_str().unwrap()).await; + assert!(matches!( + result.expect_err("nonexistent should fail"), + LocalFileSystemError::NotFound { .. } + )); + + // successful move A to C + assert!(a.exists()); + assert!(!c.exists()); + match rename_noreplace(a.to_str().unwrap(), c.to_str().unwrap()).await { + Err(LocalFileSystemError::InvalidArgument {source, ..}) => + panic!("expected success, got: {:?}. Note: atomically renaming Windows files from WSL2 is not supported.", source), + Err(e) => panic!("expected success, got: {:?}", e), + _ => {} + } + assert!(!a.exists()); + assert!(c.exists()); + + // unsuccessful move B to C, C already exists, B is not deleted + assert!(b.exists()); + match rename_noreplace(b.to_str().unwrap(), c.to_str().unwrap()).await { + Err(LocalFileSystemError::AlreadyExists { path, .. }) => { + assert_eq!(path, c.to_str().unwrap()) + } + _ => panic!("unexpected"), + } + assert!(b.exists()); + assert_eq!(std::fs::read_to_string(c).unwrap(), "a"); + } + + fn create_file(dir: &Path, name: &str) -> PathBuf { + let path = dir.join(name); + let mut file = File::create(&path).unwrap(); + file.write_all(name.as_bytes()).unwrap(); + path + } +} diff --git a/rust/src/storage/file/mod.rs b/rust/src/storage/file/mod.rs deleted file mode 100644 index e33e4ce1a9..0000000000 --- a/rust/src/storage/file/mod.rs +++ /dev/null @@ -1,289 +0,0 @@ -//! Local file storage backend. This backend read and write objects from local filesystem. -//! -//! The local file storage backend is multi-writer safe. - -use super::{ObjectMeta, StorageBackend, StorageError}; -use chrono::DateTime; -use futures::{stream::BoxStream, StreamExt}; -use std::collections::VecDeque; -use std::io; -use std::io::SeekFrom; -use std::ops::Range; -use std::path::Path; -use tokio::fs; -use tokio::io::{AsyncReadExt, AsyncSeekExt, AsyncWriteExt}; -use uuid::Uuid; -use walkdir::WalkDir; - -mod rename; - -/// Multi-writer support for different platforms: -/// -/// * Modern Linux kernels are well supported. However because Linux implementation leverages -/// `RENAME_NOREPLACE`, older versions of the kernel might not work depending on what filesystem is -/// being used: -/// * ext4 requires >= Linux 3.15 -/// * btrfs, shmem, and cif requires >= Linux 3.17 -/// * xfs requires >= Linux 4.0 -/// * ext2, minix, reiserfs, jfs, vfat, and bpf requires >= Linux 4.9 -/// * Darwin is supported but not fully tested. -/// Patches welcome. -/// * Support for other platforms are not implemented at the moment. -#[derive(Default, Debug)] -pub struct FileStorageBackend { - #[allow(dead_code)] - root: String, -} - -impl FileStorageBackend { - /// Creates a new FileStorageBackend. - pub fn new(root: &str) -> Self { - Self { - root: String::from(root), - } - } -} - -#[async_trait::async_trait] -impl StorageBackend for FileStorageBackend { - async fn head_obj(&self, path: &str) -> Result { - let attr = fs::metadata(path).await?; - - Ok(ObjectMeta { - path: path.to_string(), - modified: DateTime::from(attr.modified().unwrap()), - size: Some(attr.len().try_into().unwrap()), - }) - } - - async fn get_obj(&self, path: &str) -> Result, StorageError> { - fs::read(path).await.map_err(StorageError::from) - } - - async fn get_range(&self, path: &str, range: Range) -> Result, StorageError> { - let mut file = fs::File::open(path).await.map_err(|e| { - if e.kind() == std::io::ErrorKind::NotFound { - StorageError::NotFound - } else { - StorageError::Generic(e.to_string()) - } - })?; - let to_read = range.end - range.start; - file.seek(SeekFrom::Start(range.start as u64)) - .await - .map_err(|e| StorageError::Generic(e.to_string()))?; - - let mut buf = Vec::with_capacity(to_read); - let _read = file - .take(to_read as u64) - .read_to_end(&mut buf) - .await - .map_err(|e| StorageError::Generic(e.to_string()))?; - - Ok(buf) - } - - async fn list_objs<'a>( - &'a self, - path: &'a str, - ) -> Result>, StorageError> { - let walkdir = WalkDir::new(path) - // Don't include the root directory itself - .min_depth(1); - - let meta_iter = walkdir.into_iter().filter_map(move |result_dir_entry| { - match convert_walkdir_result(result_dir_entry) { - Err(e) => Some(Err(e)), - Ok(None) => None, - Ok(entry @ Some(_)) => entry - .filter(|dir_entry| dir_entry.file_type().is_file()) - .map(|entry| { - let file_path = - String::from(entry.path().to_str().ok_or_else(|| { - StorageError::Generic("invalid path".to_string()) - })?); - match entry.metadata() { - Ok(meta) => Ok(ObjectMeta { - path: file_path, - modified: meta.modified()?.into(), - size: Some(meta.len().try_into().map_err(|_| { - StorageError::Generic("cannot convert to i64".to_string()) - })?), - }), - Err(err) - if err.io_error().map(|e| e.kind()) - == Some(io::ErrorKind::NotFound) => - { - Err(StorageError::NotFound) - } - Err(err) => Err(StorageError::WalkDir { source: err }), - } - }), - } - }); - - // list in batches of CHUNK_SIZE - const CHUNK_SIZE: usize = 1024; - - let buffer = VecDeque::with_capacity(CHUNK_SIZE); - let stream = futures::stream::try_unfold( - (meta_iter, buffer), - |(mut meta_iter, mut buffer)| async move { - if buffer.is_empty() { - (meta_iter, buffer) = tokio::task::spawn_blocking(move || { - for _ in 0..CHUNK_SIZE { - match meta_iter.next() { - Some(r) => buffer.push_back(r), - None => break, - } - } - (meta_iter, buffer) - }) - .await - .map_err(|err| StorageError::Generic(err.to_string()))?; - } - - match buffer.pop_front() { - Some(Err(e)) => Err(e), - Some(Ok(meta)) => Ok(Some((meta, (meta_iter, buffer)))), - None => Ok(None), - } - }, - ); - - Ok(stream.boxed()) - } - - async fn put_obj(&self, path: &str, obj_bytes: &[u8]) -> Result<(), StorageError> { - if let Some(parent) = Path::new(path).parent() { - fs::create_dir_all(parent).await?; - } - let tmp_path = &format!("{}_{}", path, Uuid::new_v4()); - let mut f = fs::OpenOptions::new() - .create(true) - .truncate(true) - .write(true) - .open(tmp_path) - .await?; - - f.write_all(obj_bytes).await?; - f.sync_all().await?; - drop(f); - - // as temp path is transparent to end user, we could use syscall directly here - match fs::rename(tmp_path, path).await { - Ok(_) => Ok(()), - Err(e) => { - // If rename failed, clean up the temp file. - self.delete_obj(tmp_path).await?; - Err(StorageError::from(e)) - } - } - } - - async fn rename_obj_noreplace(&self, src: &str, dst: &str) -> Result<(), StorageError> { - rename::rename_noreplace(src, dst).await - } - - async fn delete_obj(&self, path: &str) -> Result<(), StorageError> { - fs::remove_file(path).await.map_err(StorageError::from) - } -} - -/// Convert walkdir results and converts not-found errors into `None`. -fn convert_walkdir_result( - res: std::result::Result, -) -> Result, StorageError> { - match res { - Ok(entry) => Ok(Some(entry)), - Err(walkdir_err) => match walkdir_err.io_error() { - Some(io_err) => match io_err.kind() { - io::ErrorKind::NotFound => Ok(None), - _ => Err(StorageError::Generic(io_err.to_string())), - }, - None => Err(StorageError::Generic(walkdir_err.to_string())), - }, - } -} - -#[cfg(test)] -mod tests { - use super::super::parse_uri; - use super::*; - - #[tokio::test] - async fn put_and_rename() { - let tmp_dir = tempdir::TempDir::new("rename_test").unwrap(); - let backend = FileStorageBackend::new(tmp_dir.path().to_str().unwrap()); - - let tmp_file_path = tmp_dir.path().join("tmp_file"); - let new_file_path = tmp_dir.path().join("new_file"); - - let tmp_file = tmp_file_path.to_str().unwrap(); - let new_file = new_file_path.to_str().unwrap(); - - // first try should result in successful rename - backend.put_obj(tmp_file, b"hello").await.unwrap(); - if let Err(e) = backend.rename_obj_noreplace(tmp_file, new_file).await { - panic!("Expect put_obj to return Ok, got Err: {:#?}", e) - } - - // second try should result in already exists error - backend.put_obj(tmp_file, b"hello").await.unwrap(); - assert!(matches!( - backend.rename_obj_noreplace(tmp_file, new_file).await, - Err(StorageError::AlreadyExists(s)) if s == new_file_path.to_str().unwrap(), - )); - } - - #[tokio::test] - async fn delete_obj() { - let tmp_dir = tempdir::TempDir::new("delete_test").unwrap(); - let tmp_file_path = tmp_dir.path().join("tmp_file"); - let backend = FileStorageBackend::new(tmp_dir.path().to_str().unwrap()); - - // put object - let path = tmp_file_path.to_str().unwrap(); - backend.put_obj(path, &[]).await.unwrap(); - assert_eq!(fs::metadata(path).await.is_ok(), true); - - // delete object - backend.delete_obj(path).await.unwrap(); - assert_eq!(fs::metadata(path).await.is_ok(), false) - } - - #[tokio::test] - async fn delete_objs() { - let tmp_dir = tempdir::TempDir::new("delete_test").unwrap(); - let tmp_file_path1 = tmp_dir.path().join("tmp_file1"); - let tmp_file_path2 = tmp_dir.path().join("tmp_file2"); - let backend = FileStorageBackend::new(tmp_dir.path().to_str().unwrap()); - - // put object - let path1 = tmp_file_path1.to_str().unwrap(); - let path2 = tmp_file_path2.to_str().unwrap(); - backend.put_obj(path1, &[]).await.unwrap(); - backend.put_obj(path2, &[]).await.unwrap(); - assert_eq!(fs::metadata(path1).await.is_ok(), true); - assert_eq!(fs::metadata(path2).await.is_ok(), true); - - // delete object - backend - .delete_objs(&[path1.to_string(), path2.to_string()]) - .await - .unwrap(); - assert_eq!(fs::metadata(path1).await.is_ok(), false); - assert_eq!(fs::metadata(path2).await.is_ok(), false) - } - - #[test] - fn test_parse_uri() { - let uri = parse_uri("foo/bar").unwrap(); - assert_eq!(uri.path(), "foo/bar"); - assert_eq!(uri.into_localpath().unwrap(), "foo/bar"); - - let uri2 = parse_uri("file:///foo/bar").unwrap(); - assert_eq!(uri2.path(), "/foo/bar"); - assert_eq!(uri2.into_localpath().unwrap(), "/foo/bar"); - } -} diff --git a/rust/src/storage/file/rename.rs b/rust/src/storage/file/rename.rs deleted file mode 100644 index d4b7c46b03..0000000000 --- a/rust/src/storage/file/rename.rs +++ /dev/null @@ -1,174 +0,0 @@ -use crate::StorageError; - -// Generic implementation (Requires 2 system calls) -#[cfg(not(any( - all(target_os = "linux", target_env = "gnu", glibc_renameat2), - target_os = "macos" -)))] -mod imp { - use super::*; - - pub async fn rename_noreplace(from: &str, to: &str) -> Result<(), StorageError> { - let from_path = String::from(from); - let to_path = String::from(to); - - tokio::task::spawn_blocking(move || { - std::fs::hard_link(&from_path, &to_path).map_err(|err| { - if err.kind() == std::io::ErrorKind::AlreadyExists { - StorageError::AlreadyExists(to_path) - } else { - err.into() - } - })?; - - std::fs::remove_file(from_path)?; - - Ok(()) - }) - .await - .unwrap() - } -} - -// Optimized implementations (Only 1 system call) -#[cfg(any( - all(target_os = "linux", target_env = "gnu", glibc_renameat2), - target_os = "macos" -))] -mod imp { - use super::*; - use std::ffi::CString; - - fn to_c_string(p: &str) -> Result { - CString::new(p).map_err(|e| StorageError::Generic(format!("{}", e))) - } - - pub async fn rename_noreplace(from: &str, to: &str) -> Result<(), StorageError> { - let cs_from = to_c_string(from)?; - let cs_to = to_c_string(to)?; - - let ret = unsafe { - tokio::task::spawn_blocking(move || { - let ret = platform_specific_rename(cs_from.as_ptr(), cs_to.as_ptr()); - if ret != 0 { - Err(errno::errno()) - } else { - Ok(()) - } - }) - .await - .unwrap() - }; - - match ret { - Err(e) => { - if let libc::EEXIST = e.0 { - return Err(StorageError::AlreadyExists(String::from(to))); - } - if let libc::EINVAL = e.0 { - return Err(StorageError::Generic(format!( - "rename_noreplace failed with message '{}'", - e - ))); - } - Err(StorageError::other_std_io_err(format!( - "failed to rename {} to {}: {}", - from, to, e - ))) - } - Ok(_) => Ok(()), - } - } - - #[allow(unused_variables)] - unsafe fn platform_specific_rename(from: *const libc::c_char, to: *const libc::c_char) -> i32 { - cfg_if::cfg_if! { - if #[cfg(all(target_os = "linux", target_env = "gnu"))] { - libc::renameat2(libc::AT_FDCWD, from, libc::AT_FDCWD, to, libc::RENAME_NOREPLACE) - } else if #[cfg(target_os = "macos")] { - libc::renamex_np(from, to, libc::RENAME_EXCL) - } else { - unreachable!() - } - } - } -} - -/// Atomically renames `from` to `to`. -/// `from` has to exist, but `to` is not, otherwise the operation will fail. -#[inline] -pub async fn rename_noreplace(from: &str, to: &str) -> Result<(), StorageError> { - imp::rename_noreplace(from, to).await -} - -#[cfg(test)] -mod tests { - use super::*; - use std::fs::File; - use std::io::Write; - use std::path::{Path, PathBuf}; - - #[tokio::test()] - async fn test_rename_noreplace() { - let tmp_dir = tempdir::TempDir::new_in(".", "test_rename_noreplace").unwrap(); - let a = create_file(&tmp_dir.path(), "a"); - let b = create_file(&tmp_dir.path(), "b"); - let c = &tmp_dir.path().join("c"); - - // unsuccessful move not_exists to C, not_exists is missing - match rename_noreplace("not_exists", c.to_str().unwrap()).await { - Err(StorageError::NotFound) => {} - Err(StorageError::Io { source: e }) => { - cfg_if::cfg_if! { - if #[cfg(target_os = "windows")] { - assert_eq!( - e.to_string(), - format!( - "failed to rename not_exists to {}: The system cannot find the file specified. (os error 2)", - c.to_str().unwrap() - ) - ); - } else { - assert_eq!( - e.to_string(), - format!( - "failed to rename not_exists to {}: No such file or directory", - c.to_str().unwrap() - ) - ); - } - } - } - Err(e) => panic!("expect std::io::Error, got: {:#}", e), - Ok(()) => panic!("{}", "expect rename to fail with Err, but got Ok"), - } - - // successful move A to C - assert!(a.exists()); - assert!(!c.exists()); - match rename_noreplace(a.to_str().unwrap(), c.to_str().unwrap()).await { - Err(StorageError::Generic(e)) if e == "rename_noreplace failed with message 'Invalid argument'" => - panic!("expected success, got: {:?}. Note: atomically renaming Windows files from WSL2 is not supported.", e), - Err(e) => panic!("expected success, got: {:?}", e), - _ => {} - } - assert!(!a.exists()); - assert!(c.exists()); - - // unsuccessful move B to C, C already exists, B is not deleted - assert!(b.exists()); - match rename_noreplace(b.to_str().unwrap(), c.to_str().unwrap()).await { - Err(StorageError::AlreadyExists(p)) => assert_eq!(p, c.to_str().unwrap()), - _ => panic!("unexpected"), - } - assert!(b.exists()); - assert_eq!(std::fs::read_to_string(c).unwrap(), "a"); - } - - fn create_file(dir: &Path, name: &str) -> PathBuf { - let path = dir.join(name); - let mut file = File::create(&path).unwrap(); - file.write_all(name.as_bytes()).unwrap(); - path - } -} diff --git a/rust/src/storage/gcs/client.rs b/rust/src/storage/gcs/client.rs deleted file mode 100644 index 58ebfc4378..0000000000 --- a/rust/src/storage/gcs/client.rs +++ /dev/null @@ -1,163 +0,0 @@ -use super::{util, GCSClientError, GCSObject}; -use futures::Stream; -use std::convert::{TryFrom, TryInto}; -use std::path::PathBuf; -/// Google Cloud Storage http client -use std::sync::Arc; -use tame_gcs::objects::{self, Object}; -use tame_oauth::gcp as oauth; - -use log::debug; - -/// Struct maintaining the state responsible for communicating -/// with the google cloud storage service -pub struct GCSStorageBackend { - /// The reqwest client used for handling http requests - pub client: reqwest::Client, - /// The path to the path to the credentials file - pub cred_path: PathBuf, - /// The handle to our oauth token - pub auth: Arc, -} - -impl std::fmt::Debug for GCSStorageBackend { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - f.debug_struct("GCSStorageBackend {...}").finish() - } -} - -impl TryFrom for GCSStorageBackend { - type Error = GCSClientError; - fn try_from(cred_path: PathBuf) -> Result { - let client = reqwest::Client::builder().build()?; - let cred_contents = std::fs::read_to_string(&cred_path) - .map_err(|source| Self::Error::CredentialsError { source })?; - let svc_account_info = oauth::ServiceAccountInfo::deserialize(cred_contents)?; - let svc_account_access = oauth::ServiceAccountAccess::new(svc_account_info)?; - - Ok(Self { - client, - cred_path, - auth: std::sync::Arc::new(svc_account_access), - }) - } -} - -impl GCSStorageBackend { - pub async fn metadata<'a>( - &self, - path: GCSObject<'_>, - ) -> Result { - debug!("creating request"); - let get_meta_request = Object::get(&path, None)?; - debug!("executing request"); - let response = - util::execute::<_, objects::GetObjectResponse>(self, get_meta_request).await?; - - debug!("returning meta"); - Ok(response.metadata) - } - - pub async fn download<'a>(&self, path: GCSObject<'_>) -> Result { - let download_request = Object::download(&path, None)?; - - let response = util::execute::<_, objects::DownloadObjectResponse>(self, download_request) - .await - .map_err(util::check_object_not_found)?; - - Ok(response.consume()) - } - - pub fn list<'a>( - &'a self, - uri: GCSObject<'a>, - ) -> impl Stream> + 'a { - let mut page_token: Option = None; - - async_stream::try_stream! { - loop { - - let list_request_opts = Some(objects::ListOptional { - prefix: Some(uri.path.as_ref()), - page_token: page_token.as_deref(), - standard_params: tame_gcs::common::StandardQueryParameters { - // We are only interested in the name and updated timestamp - // to subsequently populate the ObjectMeta struct - fields: Some("items(name, updated"), - ..Default::default() - }, - ..Default::default() - }); - - let list_request = Object::list(&uri.bucket, list_request_opts)?; - let list_response = util::execute::<_, objects::ListResponse>( - self, list_request).await?; - - for object_meta in list_response.objects { - yield object_meta - } - - // If we have a page token it means there may be more items - // that fulfill the parameters - page_token = list_response.page_token; - if page_token.is_none() { - break; - } - } - } - } - - pub async fn insert<'a, 'b>( - &self, - uri: GCSObject<'a>, - content: Vec, - ) -> Result<(), GCSClientError> { - let content_len = content.len().try_into().unwrap(); - let content_body = std::io::Cursor::new(content); - - let insert_request = Object::insert_simple(&uri, content_body, content_len, None)?; - let _response = util::execute::<_, objects::InsertResponse>(self, insert_request).await?; - - Ok(()) - } - - pub async fn rename_noreplace<'a>( - &self, - src: GCSObject<'a>, - dst: GCSObject<'a>, - ) -> Result<(), GCSClientError> { - let mut rewrite_token = None; - - loop { - let metadata = None; - let precondition = Some(objects::RewriteObjectOptional { - destination_conditionals: Some(tame_gcs::common::Conditionals { - if_generation_match: Some(0), - ..Default::default() - }), - ..Default::default() - }); - - let rewrite_http_request = - Object::rewrite(&src, &dst, rewrite_token, metadata, precondition)?; - let response = - util::execute::<_, objects::RewriteObjectResponse>(self, rewrite_http_request) - .await - .map_err(util::check_precondition_status)?; - - rewrite_token = response.rewrite_token; - if rewrite_token.is_none() { - break; - } - } - - self.delete(src).await - } - - pub async fn delete<'a>(&self, uri: GCSObject<'_>) -> Result<(), GCSClientError> { - let delete_request = Object::delete(&uri, None)?; - let _response = - util::execute::<_, objects::DeleteObjectResponse>(self, delete_request).await?; - Ok(()) - } -} diff --git a/rust/src/storage/gcs/error.rs b/rust/src/storage/gcs/error.rs deleted file mode 100644 index 782d370512..0000000000 --- a/rust/src/storage/gcs/error.rs +++ /dev/null @@ -1,46 +0,0 @@ -/// Error enum that represents an issue encountered -/// during interaction with the GCS service -#[derive(thiserror::Error, Debug)] -pub enum GCSClientError { - #[error("Authentication error: {source}")] - AuthError { - #[from] - source: tame_oauth::Error, - }, - - #[error("Error interacting with GCS: {source}")] - GCSError { - #[from] - source: tame_gcs::Error, - }, - - #[error("Reqwest error: {source}")] - ReqwestError { - #[from] - source: reqwest::Error, - }, - - #[error("IO error: {source}")] - IOError { - #[from] - source: std::io::Error, - }, - - #[error("HTTP error: {source}")] - HttpError { - #[from] - source: tame_gcs::http::Error, - }, - - #[error("Resource Not Found")] - NotFound, - - #[error("Precondition Failed")] - PreconditionFailed, - - #[error("Error: {0}")] - Other(String), - - #[error("Credentials error: {source}")] - CredentialsError { source: std::io::Error }, -} diff --git a/rust/src/storage/gcs/mod.rs b/rust/src/storage/gcs/mod.rs deleted file mode 100644 index 693bd27c66..0000000000 --- a/rust/src/storage/gcs/mod.rs +++ /dev/null @@ -1,118 +0,0 @@ -//! Google Cloud Storage backend. -//! -//! This module is gated behind the "gcs" feature. Its usage also requires -//! the `SERVICE_ACCOUNT` environment variables to be set to the path of -//! credentials with permission to read from the bucket. - -mod client; -mod error; -mod object; -mod util; - -// Exports -pub(crate) use client::GCSStorageBackend; -pub(crate) use error::GCSClientError; -pub(crate) use object::GCSObject; - -use futures::stream::BoxStream; -use std::convert::TryInto; -use std::ops::Range; - -use log::debug; - -use super::{parse_uri, ObjectMeta, StorageBackend, StorageError}; - -impl GCSStorageBackend { - pub(crate) fn new() -> Result { - let cred_path = std::env::var("SERVICE_ACCOUNT") - .map(std::path::PathBuf::from) - .map_err(|_err| { - StorageError::GCSConfig( - "SERVICE_ACCOUNT environment variable must be set".to_string(), - ) - })?; - - Ok(cred_path.try_into()?) - } -} - -impl From for ObjectMeta { - fn from(metadata: tame_gcs::objects::Metadata) -> ObjectMeta { - ObjectMeta { - path: metadata.name.unwrap(), - modified: metadata.updated.unwrap(), - size: metadata.size.map(|s| s.try_into().unwrap()), - } - } -} - -#[async_trait::async_trait] -impl StorageBackend for GCSStorageBackend { - /// Fetch object metadata without reading the actual content - async fn head_obj(&self, path: &str) -> Result { - debug!("getting meta for: {}", path); - let obj_uri = parse_uri(path)?.into_gcs_object()?; - let metadata = self.metadata(obj_uri).await?; - Ok(metadata.into()) - } - - /// Fetch object content - async fn get_obj(&self, path: &str) -> Result, StorageError> { - debug!("getting object at: {}", path); - let obj_uri = parse_uri(path)?.into_gcs_object()?; - match self.download(obj_uri).await { - Err(GCSClientError::NotFound) => return Err(StorageError::NotFound), - res => Ok(res?.to_vec()), - } - } - - async fn get_range(&self, _path: &str, _range: Range) -> Result, StorageError> { - todo!("get range not implemented for gcs") - } - - /// Return a list of objects by `path` prefix in an async stream. - async fn list_objs<'a>( - &'a self, - path: &'a str, - ) -> Result>, StorageError> { - let prefix = parse_uri(path)?.into_gcs_object()?; - let obj_meta_stream = async_stream::stream! { - for await meta in self.list(prefix) { - let obj_meta = meta?; - yield Ok(obj_meta.into()); - } - }; - - Ok(Box::pin(obj_meta_stream)) - } - - /// Create new object with `obj_bytes` as content. - async fn put_obj(&self, path: &str, obj_bytes: &[u8]) -> Result<(), StorageError> { - let dst = parse_uri(path)?.into_gcs_object()?; - Ok(self.insert(dst, obj_bytes.to_vec()).await?) - } - - /// Moves object from `src` to `dst`. - /// - /// Implementation note: - /// - /// For a multi-writer safe backend, `rename_obj` needs to implement `rename if not exists` semantic. - /// In other words, if the destination path already exists, rename should return a - /// [StorageError::AlreadyExists] error. - async fn rename_obj_noreplace(&self, src: &str, dst: &str) -> Result<(), StorageError> { - let src_uri = parse_uri(src)?.into_gcs_object()?; - let dst_uri = parse_uri(dst)?.into_gcs_object()?; - match self.rename_noreplace(src_uri, dst_uri).await { - Err(GCSClientError::PreconditionFailed) => { - return Err(StorageError::AlreadyExists(dst.to_string())) - } - res => Ok(res?), - } - } - - /// Deletes object by `path`. - async fn delete_obj(&self, path: &str) -> Result<(), StorageError> { - let uri = parse_uri(path)?.into_gcs_object()?; - Ok(self.delete(uri).await?) - } -} diff --git a/rust/src/storage/gcs/object.rs b/rust/src/storage/gcs/object.rs deleted file mode 100644 index fca3838699..0000000000 --- a/rust/src/storage/gcs/object.rs +++ /dev/null @@ -1,41 +0,0 @@ -use std::fmt; - -/// Struct describing an object stored in GCS. -#[derive(Debug)] -pub struct GCSObject<'a> { - /// The bucket where the object is stored. - pub bucket: tame_gcs::BucketName<'a>, - /// The path of the object within the bucket. - pub path: tame_gcs::ObjectName<'a>, -} - -impl<'a> GCSObject<'a> { - //// Create a new GCSObject from a bucket and path. - pub(crate) fn new(bucket: &'a str, path: &'a str) -> Self { - // We do not validate the input strings here - // as it is expected that they are correctly parsed and validated a level up in the - // storage module - GCSObject { - bucket: tame_gcs::BucketName::non_validated(bucket), - path: tame_gcs::ObjectName::non_validated(path), - } - } -} - -impl<'a> fmt::Display for GCSObject<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "gs://{}/{}", self.bucket, self.path) - } -} - -impl<'a> AsRef> for GCSObject<'a> { - fn as_ref(&self) -> &tame_gcs::BucketName<'a> { - &self.bucket - } -} - -impl<'a> AsRef> for GCSObject<'a> { - fn as_ref(&self) -> &tame_gcs::ObjectName<'a> { - &self.path - } -} diff --git a/rust/src/storage/gcs/util.rs b/rust/src/storage/gcs/util.rs deleted file mode 100644 index 295e3d164c..0000000000 --- a/rust/src/storage/gcs/util.rs +++ /dev/null @@ -1,150 +0,0 @@ -use super::{GCSClientError, GCSStorageBackend}; -/// This code is largely duplicated from https://github.com/EmbarkStudios/gsutil -use bytes::BufMut; -use futures::StreamExt; -use std::convert::TryInto; -use std::iter::Iterator; -use tame_gcs::http; -use tame_oauth::gcp as oauth; - -async fn get_token(backend: &GCSStorageBackend) -> Result { - Ok( - match backend.auth.get_token(&[tame_gcs::Scopes::ReadWrite])? { - oauth::TokenOrRequest::Token(token) => token, - oauth::TokenOrRequest::Request { - request, - scope_hash, - .. - } => { - let (parts, body) = request.into_parts(); - let read_body = std::io::Cursor::new(body); - let new_request = http::Request::from_parts(parts, read_body); - - let req = convert_request(new_request, &backend.client).await?; - let res = backend.client.execute(req).await?; - let response = convert_response(res).await?; - backend.auth.parse_token_response(scope_hash, response)? - } - }, - ) -} - -/// Converts a vanilla `http::Request` into a `reqwest::Request` -async fn convert_request( - req: http::Request, - client: &reqwest::Client, -) -> Result -where - B: std::io::Read + Send + 'static, -{ - let (parts, mut body) = req.into_parts(); - - let uri = parts.uri.to_string(); - - let builder = match parts.method { - http::Method::GET => client.get(&uri), - http::Method::POST => client.post(&uri), - http::Method::DELETE => client.delete(&uri), - http::Method::PATCH => client.patch(&uri), - http::Method::PUT => client.put(&uri), - method => panic!("Invalid http method: {}", method), - }; - - let content_len = tame_gcs::util::get_content_length(&parts.headers).unwrap_or(0); - let mut buffer = bytes::BytesMut::with_capacity(content_len); - - let mut block = [0u8; 8 * 1024]; - - loop { - let read = body.read(&mut block)?; - - if read > 0 { - buffer.extend_from_slice(&block[..read]); - } else { - break; - } - } - - Ok(builder - .header(reqwest::header::CONTENT_LENGTH, content_len) - .headers(parts.headers) - .body(buffer.freeze()) - .build()?) -} - -/// Converts a `reqwest::Response` into a vanilla `http::Response`. This currently copies -/// the entire response body into a single buffer with no streaming -async fn convert_response( - res: reqwest::Response, -) -> Result, GCSClientError> { - let mut builder = http::Response::builder() - .status(res.status()) - .version(res.version()); - - let headers = builder - .headers_mut() - .ok_or_else(|| GCSClientError::Other("failed to convert response headers".to_string()))?; - - headers.extend( - res.headers() - .into_iter() - .map(|(k, v)| (k.clone(), v.clone())), - ); - - let content_len = tame_gcs::util::get_content_length(headers).unwrap_or_default(); - let mut buffer = bytes::BytesMut::with_capacity(content_len); - - let mut stream = res.bytes_stream(); - - while let Some(item) = stream.next().await { - buffer.put(item?); - } - - Ok(builder.body(buffer.freeze())?) -} - -/// Executes a GCS request via a reqwest client and returns the parsed response/API error -pub async fn execute( - ctx: &GCSStorageBackend, - mut req: http::Request, -) -> Result -where - R: tame_gcs::ApiResponse, - B: std::io::Read + Send + 'static, -{ - // First, get our oauth token, which can mean we have to do an additional - // request if we've never retrieved one yet, or the one we are using has expired - let token = get_token(ctx).await?; - - // Add the authorization token, note that the tame-oauth crate will automatically - // set the HeaderValue correctly, in the GCP case this is usually "Bearer " - req.headers_mut() - .insert(http::header::AUTHORIZATION, token.try_into()?); - - let request = convert_request(req, &ctx.client).await?; - let response = ctx.client.execute(request).await?; - let response = convert_response(response).await?; - - Ok(R::try_from_parts(response)?) -} - -use http::status::StatusCode; -use tame_gcs::error::HttpStatusError; -pub fn check_object_not_found(err: GCSClientError) -> GCSClientError { - match err { - GCSClientError::GCSError { - source: tame_gcs::error::Error::HttpStatus(HttpStatusError(StatusCode::NOT_FOUND)), - } => GCSClientError::NotFound, - err => err, - } -} - -pub fn check_precondition_status(err: GCSClientError) -> GCSClientError { - match err { - GCSClientError::GCSError { - source: - tame_gcs::error::Error::HttpStatus(HttpStatusError(StatusCode::PRECONDITION_FAILED)), - } => GCSClientError::PreconditionFailed, - err => err, - } -} diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index 3394e1906c..4472e517c6 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -1,600 +1,312 @@ //! Object storage backend abstraction layer for Delta Table transaction logs and data -#[cfg(feature = "azure")] -use azure_core::error::{Error as AzureError, ErrorKind as AzureErrorKind}; -use chrono::{DateTime, Utc}; -use futures::stream::BoxStream; -#[cfg(any(feature = "s3", feature = "s3-rustls"))] -use hyper::http::uri::InvalidUri; -use object_store::Error as ObjectStoreError; -use std::collections::HashMap; -use std::fmt::Debug; -use std::ops::Range; -use std::sync::Arc; -use walkdir::Error as WalkDirError; - -#[cfg(feature = "azure")] -pub mod azure; pub mod file; -#[cfg(any(feature = "gcs"))] -pub mod gcs; #[cfg(any(feature = "s3", feature = "s3-rustls"))] pub mod s3; -/// Error enum that represents an invalid URI. -#[derive(thiserror::Error, Debug, PartialEq, Eq)] -pub enum UriError { - /// Error returned when the URI contains a scheme that is not handled. - #[error("Invalid URI scheme: {0}")] - InvalidScheme(String), - /// Error returned when a local file system path is expected, but the URI is not a local file system path. - #[error("Expected local path URI, found: {0}")] - ExpectedSLocalPathUri(String), - - /// Error returned when the URI is expected to be an object storage path, but does not include a bucket part. - #[cfg(any(feature = "gcs", feature = "s3", feature = "s3-rustls"))] - #[error("Object URI missing bucket")] - MissingObjectBucket, - /// Error returned when the URI is expected to be an object storage path, but does not include a key part. - #[cfg(any(feature = "gcs", feature = "s3", feature = "s3-rustls"))] - #[error("Object URI missing key")] - MissingObjectKey, - /// Error returned when an S3 path is expected, but the URI is not an S3 URI. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Expected S3 URI, found: {0}")] - ExpectedS3Uri(String), - - /// Error returned when an GCS path is expected, but the URI is not an GCS URI. - #[cfg(feature = "gcs")] - #[error("Expected GCS URI, found: {0}")] - ExpectedGCSUri(String), - - /// Error returned when an Azure URI is expected, but the URI is not an Azure URI. - #[cfg(feature = "azure")] - #[error("Expected Azure URI, found: {0}")] - ExpectedAzureUri(String), +use crate::builder::StorageUrl; +use bytes::Bytes; +#[cfg(feature = "datafusion-ext")] +use datafusion::datasource::object_store::ObjectStoreUrl; +use futures::{stream::BoxStream, StreamExt, TryStreamExt}; +use lazy_static::lazy_static; +use object_store::{ + path::{Path, DELIMITER}, + DynObjectStore, Error as ObjectStoreError, GetResult, ListResult, MultipartId, ObjectMeta, + ObjectStore, Result as ObjectStoreResult, +}; +use std::ops::Range; +use std::sync::Arc; +use tokio::io::AsyncWrite; - /// Error returned when an Azure URI is expected, but the URI is missing the scheme. - #[cfg(feature = "azure")] - #[error("Object URI missing filesystem")] - MissingObjectFileSystem, - /// Error returned when an Azure URI is expected, but the URI is missing the account name and - /// path. - #[cfg(feature = "azure")] - #[error("Object URI missing account name and path")] - MissingObjectAccount, - /// Error returned when an Azure URI is expected, but the URI is missing the account name. - #[cfg(feature = "azure")] - #[error("Object URI missing account name")] - MissingObjectAccountName, - /// Error returned when an Azure URI is expected, but the URI is missing the path. - #[cfg(feature = "azure")] - #[error("Object URI missing path")] - MissingObjectPath, - /// Error returned when container in an Azure URI doesn't match the expected value - #[cfg(feature = "azure")] - #[error("Container mismatch, expected: {expected}, got: {got}")] - ContainerMismatch { - /// Expected container value - expected: String, - /// Actual container value - got: String, - }, +lazy_static! { + static ref DELTA_LOG_PATH: Path = Path::from("_delta_log"); } -/// Enum with variants representing each supported storage backend. -#[derive(Debug)] -pub enum Uri<'a> { - /// URI for local file system backend. - LocalPath(&'a str), - /// URI for S3 backend. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - S3Object(s3::S3Object<'a>), - /// URI for Azure backend. - #[cfg(feature = "azure")] - AdlsGen2Object(azure::AdlsGen2Object<'a>), - /// URI for GCS backend - #[cfg(feature = "gcs")] - GCSObject(gcs::GCSObject<'a>), +/// Configuration for a DeltaObjectStore +#[derive(Debug, Clone)] +struct DeltaObjectStoreConfig { + storage_url: StorageUrl, } -impl<'a> Uri<'a> { - /// Converts the URI to an S3Object. Returns UriError if the URI is not valid for the S3 - /// backend. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - pub fn into_s3object(self) -> Result, UriError> { - match self { - Uri::S3Object(x) => Ok(x), - #[cfg(feature = "azure")] - Uri::AdlsGen2Object(x) => Err(UriError::ExpectedS3Uri(x.to_string())), - #[cfg(feature = "gcs")] - Uri::GCSObject(x) => Err(UriError::ExpectedS3Uri(x.to_string())), - Uri::LocalPath(x) => Err(UriError::ExpectedS3Uri(x.to_string())), - } +impl DeltaObjectStoreConfig { + /// Create a new [DeltaObjectStoreConfig] + pub fn new(storage_url: StorageUrl) -> Self { + Self { storage_url } } - /// Converts the URI to an AdlsGen2Object. Returns UriError if the URI is not valid for the - /// Azure backend. - #[cfg(feature = "azure")] - pub fn into_adlsgen2_object(self) -> Result, UriError> { - match self { - Uri::AdlsGen2Object(x) => Ok(x), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - Uri::S3Object(x) => Err(UriError::ExpectedAzureUri(x.to_string())), - #[cfg(feature = "gcs")] - Uri::GCSObject(x) => Err(UriError::ExpectedAzureUri(x.to_string())), - Uri::LocalPath(x) => Err(UriError::ExpectedAzureUri(x.to_string())), - } + /// Prefix a path with the table root path + fn full_path(&self, location: &Path) -> ObjectStoreResult { + let path: &str = location.as_ref(); + let stripped = match self.storage_url.prefix.as_ref() { + "" => path.to_string(), + p => format!("{}/{}", p, path), + }; + Ok(Path::parse(stripped.trim_end_matches(DELIMITER))?) } - /// Converts the URI to an GCSObject. Returns UriError if the URI is not valid for the - /// Google Cloud Storage backend. - #[cfg(feature = "gcs")] - pub fn into_gcs_object(self) -> Result, UriError> { - match self { - Uri::GCSObject(x) => Ok(x), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - Uri::S3Object(x) => Err(UriError::ExpectedGCSUri(x.to_string())), - #[cfg(feature = "azure")] - Uri::AdlsGen2Object(x) => Err(UriError::ExpectedGCSUri(x.to_string())), - Uri::LocalPath(x) => Err(UriError::ExpectedGCSUri(x.to_string())), - } + fn strip_prefix(&self, path: &Path) -> Option { + let path: &str = path.as_ref(); + let stripped = match self.storage_url.prefix.as_ref() { + "" => path, + p => path.strip_prefix(p)?.strip_prefix(DELIMITER)?, + }; + Path::parse(stripped).ok() } - /// Converts the URI to an str representing a local file system path. Returns UriError if the - /// URI is not valid for the file storage backend. - pub fn into_localpath(self) -> Result<&'a str, UriError> { - match self { - Uri::LocalPath(x) => Ok(x), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - Uri::S3Object(x) => Err(UriError::ExpectedSLocalPathUri(format!("{}", x))), - #[cfg(feature = "azure")] - Uri::AdlsGen2Object(x) => Err(UriError::ExpectedSLocalPathUri(format!("{}", x))), - #[cfg(feature = "gcs")] - Uri::GCSObject(x) => Err(UriError::ExpectedSLocalPathUri(format!("{}", x))), - } - } - - /// Return URI path component as String - #[inline] - pub fn path(&self) -> String { - match self { - Uri::LocalPath(x) => x.to_string(), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - Uri::S3Object(x) => x.key.to_string(), - #[cfg(feature = "azure")] - Uri::AdlsGen2Object(x) => x.path.to_string(), - #[cfg(feature = "gcs")] - Uri::GCSObject(x) => x.path.to_string(), - } + /// convert a table [Path] to a fully qualified uri + pub fn to_uri(&self, location: &Path) -> String { + let uri = match self.storage_url.scheme() { + "file" | "" => { + // On windows the drive (e.g. 'c:') is part of root and must not be prefixed. + #[cfg(windows)] + let os_uri = format!("{}/{}", self.storage_url.prefix, location.as_ref()); + #[cfg(unix)] + let os_uri = format!("/{}/{}", self.storage_url.prefix, location.as_ref()); + os_uri + } + _ => format!("{}/{}", self.storage_url.as_str(), location.as_ref()), + }; + uri.trim_end_matches('/').into() } } -/// Parses the URI and returns a variant of the Uri enum for the appropriate storage backend based -/// on scheme. -pub fn parse_uri<'a>(path: &'a str) -> Result, UriError> { - let parts: Vec<&'a str> = path.split("://").collect(); +/// Object Store implementation for DeltaTable. +/// +/// The [DeltaObjectStore] implements the [object_store::ObjectStore] trait to facilitate +/// interoperability with the larger rust / arrow ecosystem. Specifically it can directly +/// be registered as store within datafusion. +/// +/// The table root is treated as the root of the object store. +/// All [Path] are reported relative to the table root. +#[derive(Debug, Clone)] +pub struct DeltaObjectStore { + storage: Arc, + config: DeltaObjectStoreConfig, +} - if parts.len() == 1 { - return Ok(Uri::LocalPath(parts[0])); +impl std::fmt::Display for DeltaObjectStore { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "DeltaObjectStore({})", self.config.storage_url.as_str()) } +} - match parts[0] { - "s3" => { - cfg_if::cfg_if! { - if #[cfg(any(feature = "s3", feature = "s3-rustls"))] { - let mut path_parts = parts[1].splitn(2, '/'); - let bucket = match path_parts.next() { - Some(x) => x, - None => { - return Err(UriError::MissingObjectBucket); - } - }; - let key = match path_parts.next() { - Some(x) => x, - None => { - return Err(UriError::MissingObjectKey); - } - }; +impl DeltaObjectStore { + /// Create new DeltaObjectStore + pub fn new(storage_url: StorageUrl, storage: Arc) -> Self { + let config = DeltaObjectStoreConfig::new(storage_url); + Self { storage, config } + } - Ok(Uri::S3Object(s3::S3Object { bucket, key })) - } else { - Err(UriError::InvalidScheme(String::from(parts[0]))) - } - } - } + /// Get a reference to the underlying storage backend + pub fn storage_backend(&self) -> Arc { + self.storage.clone() + } - // This can probably be refactored into the above match arm - "gs" => { - cfg_if::cfg_if! { - if #[cfg(any(feature = "gcs"))] { - let mut path_parts = parts[1].splitn(2, '/'); - let bucket = match path_parts.next() { - Some(x) => x, - None => { - return Err(UriError::MissingObjectBucket); - } - }; - let path = match path_parts.next() { - Some(x) => x, - None => { - return Err(UriError::MissingObjectKey); - } - }; + /// Get fully qualified uri for table root + pub fn root_uri(&self) -> String { + self.config.to_uri(&Path::from("")) + } - Ok(Uri::GCSObject(gcs::GCSObject::new(bucket, path))) - } else { - Err(UriError::InvalidScheme(String::from(parts[0]))) - } - } - } + #[cfg(feature = "datafusion-ext")] + /// generate a unique enough url to identify the store in datafusion. + pub(crate) fn object_store_url(&self) -> ObjectStoreUrl { + // we are certain, that the URL can be parsed, since + // we make sure when we are parsing the table uri + ObjectStoreUrl::parse(format!( + "delta-rs://{}", + // NOTE We need to also replace colons, but its fine, since it just needs + // to be a unique-ish identifier for the object store in datafusion + self.config + .storage_url + .prefix + .as_ref() + .replace(DELIMITER, "-") + .replace(':', "-") + )) + .expect("Invalid object store url.") + } - "file" => Ok(Uri::LocalPath(parts[1])), + /// [Path] to Delta log + pub fn log_path(&self) -> &Path { + &DELTA_LOG_PATH + } - // Azure Data Lake Storage Gen2 - // This URI syntax is an invention of delta-rs. - // ABFS URIs should not be used since delta-rs doesn't use the Hadoop ABFS driver. - "adls2" => { - cfg_if::cfg_if! { - if #[cfg(feature = "azure")] { - let mut path_parts = parts[1].splitn(3, '/'); - let account_name = match path_parts.next() { - Some(x) => x, - None => { - return Err(UriError::MissingObjectAccount); - } - }; - let file_system = match path_parts.next() { - Some(x) => x, - None => { - return Err(UriError::MissingObjectFileSystem); - } - }; - let path = path_parts.next().unwrap_or("/"); + /// [Path] to Delta log + pub fn to_uri(&self, location: &Path) -> String { + self.config.to_uri(location) + } - Ok(Uri::AdlsGen2Object(azure::AdlsGen2Object { account_name, file_system, path })) - } else { - Err(UriError::InvalidScheme(String::from(parts[0]))) - } + /// Deletes object by `paths`. + pub async fn delete_batch(&self, paths: &[Path]) -> ObjectStoreResult<()> { + for path in paths { + match self.delete(path).await { + Ok(_) => continue, + Err(ObjectStoreError::NotFound { .. }) => continue, + Err(e) => return Err(e), } } - _ => Err(UriError::InvalidScheme(String::from(parts[0]))), + Ok(()) } } -/// Error enum returned when storage backend interaction fails. -#[derive(thiserror::Error, Debug)] -pub enum StorageError { - /// The requested object does not exist. - #[error("Object not found")] - NotFound, - /// The object written to the storage backend already exists. - /// This error is expected in some cases. - /// For example, optimistic concurrency writes between multiple processes expect to compete - /// for the same URI when writing to _delta_log. - #[error("Object exists already at path: {0}")] - AlreadyExists(String), - /// An IO error occurred while reading from the local file system. - #[error("Failed to read local object content: {source}")] - Io { - /// The raw error returned when trying to read the local file. - source: std::io::Error, - }, - - #[error("Failed to walk directory: {source}")] - /// Error raised when failing to traverse a directory - WalkDir { - /// The raw error returned when trying to read the local file. - #[from] - source: WalkDirError, - }, - /// The file system represented by the scheme is not known. - #[error("File system not supported")] - FileSystemNotSupported, - /// Wraps a generic storage backend error. The wrapped string contains the details. - #[error("Generic error: {0}")] - Generic(String), - - /// Error representing an S3 GET failure. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to read S3 object content: {source}")] - S3Get { - /// The underlying Rusoto S3 error. - source: rusoto_core::RusotoError, - }, - /// Error representing a failure when executing an S3 HEAD request. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to read S3 object metadata: {source}")] - S3Head { - /// The underlying Rusoto S3 error. - source: rusoto_core::RusotoError, - }, - /// Error representing a failure when executing an S3 list operation. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to list S3 objects: {source}")] - S3List { - /// The underlying Rusoto S3 error. - source: rusoto_core::RusotoError, - }, - /// Error representing a failure when executing an S3 PUT request. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to put S3 object: {source}")] - S3Put { - /// The underlying Rusoto S3 error. - source: rusoto_core::RusotoError, - }, - /// Error representing a failure when executing an S3 DeleteObject request. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to delete S3 object: {source}")] - S3Delete { - /// The underlying Rusoto S3 error. - #[from] - source: rusoto_core::RusotoError, - }, - /// Error representing a failure when executing an S3 DeleteObjects request. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to delete S3 object: {source}")] - S3BatchDelete { - /// The underlying Rusoto S3 error. - #[from] - source: rusoto_core::RusotoError, - }, - /// Error representing a failure when copying a S3 object - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to copy S3 object: {source}")] - S3Copy { - /// The underlying Rusoto S3 error. - source: rusoto_core::RusotoError, - }, - /// Error returned when S3 object get response contains empty body - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("S3 Object missing body content: {0}")] - S3MissingObjectBody(String), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - /// Represents a generic S3 error. The wrapped error string describes the details. - #[error("S3 error: {0}")] - S3Generic(String), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - /// Wraps the DynamoDB error - #[error("DynamoDB error: {source}")] - DynamoDb { - /// Wrapped DynamoDB error - #[from] - source: dynamodb_lock::DynamoError, - }, - /// Error representing a failure to retrieve AWS credentials. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to retrieve AWS credentials: {source}")] - AWSCredentials { - /// The underlying Rusoto CredentialsError - #[from] - source: rusoto_credential::CredentialsError, - }, - /// Error caused by the http request dispatcher not being able to be created. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Failed to create request dispatcher: {source}")] - AWSHttpClient { - /// The underlying Rusoto TlsError - #[from] - source: rusoto_core::request::TlsError, - }, - - /// Azure error - #[cfg(feature = "azure")] - #[error("Error interacting with Azure: {source}")] - Azure { - /// Azure error reason - source: AzureError, - }, - /// Azure config error - #[cfg(feature = "azure")] - #[error("Azure config error: {0}")] - AzureConfig(String), - /// Azure credentials error - #[cfg(feature = "azure")] - #[error("Azure credentials error: {source}")] - AzureCredentials { - /// Azure error reason - source: AzureError, - }, - - /// GCS config error - #[cfg(feature = "gcs")] - #[error("GCS config error: {0}")] - GCSConfig(String), - - /// GCS client error - #[cfg(feature = "gcs")] - #[error("GCS error: {source}")] - GCSError { - /// The underlying Google Cloud Error - #[from] - source: gcs::GCSClientError, - }, - - /// Error returned when the URI is invalid. - /// The wrapped UriError contains additional details. - #[error("Invalid object URI")] - Uri { - #[from] - /// Uri error details when the URI is invalid. - source: UriError, - }, - - /// Error returned when the URI is invalid. - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - #[error("Invalid URI parsing")] - ParsingUri { - #[from] - /// Uri error details when the URI parsing is invalid. - source: InvalidUri, - }, +#[async_trait::async_trait] +impl ObjectStore for DeltaObjectStore { + /// Save the provided bytes to the specified location. + async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult<()> { + let full_path = self.config.full_path(location)?; + self.storage.put(&full_path, bytes).await + } - /// underlying object store returned an error. - #[error("ObjectStore interaction failed: {source}")] - ObjectStore { - /// The wrapped [`ObjectStoreError`] - #[from] - source: ObjectStoreError, - }, -} + /// Return the bytes that are stored at the specified location. + async fn get(&self, location: &Path) -> ObjectStoreResult { + let full_path = self.config.full_path(location)?; + self.storage.get(&full_path).await + } -impl StorageError { - /// Creates a StorageError::Io error wrapping the provided error string. - pub fn other_std_io_err(desc: String) -> Self { - Self::Io { - source: std::io::Error::new(std::io::ErrorKind::Other, desc), - } + /// Return the bytes that are stored at the specified location + /// in the given byte range + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + let full_path = self.config.full_path(location)?; + object_store::ObjectStore::get_range(self.storage.as_ref(), &full_path, range).await } -} -impl From for StorageError { - fn from(error: std::io::Error) -> Self { - match error.kind() { - std::io::ErrorKind::NotFound => StorageError::NotFound, - _ => StorageError::Io { source: error }, - } + /// Return the metadata for the specified location + async fn head(&self, location: &Path) -> ObjectStoreResult { + let full_path = self.config.full_path(location)?; + self.storage.head(&full_path).await.map(|meta| ObjectMeta { + last_modified: meta.last_modified, + size: meta.size, + location: self + .config + .strip_prefix(&meta.location) + .unwrap_or(meta.location), + }) } -} -#[cfg(feature = "azure")] -impl From for StorageError { - fn from(error: AzureError) -> Self { - match error.kind() { - AzureErrorKind::HttpResponse { status, .. } if *status == 404 => StorageError::NotFound, - AzureErrorKind::HttpResponse { status, .. } if *status == 401 || *status == 403 => { - StorageError::AzureCredentials { source: error } - } - _ => StorageError::Azure { source: error }, - } + /// Delete the object at the specified location. + async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { + let full_path = self.config.full_path(location)?; + self.storage.delete(&full_path).await } -} -/// Describes metadata of a storage object. -#[derive(Debug)] -pub struct ObjectMeta { - /// The path where the object is stored. This is the path component of the object URI. - /// - /// For example: - /// * path for `s3://bucket/foo/bar` should be `foo/bar`. - /// * path for `dir/foo/bar` should be `dir/foo/bar`. + /// List all the objects with the given prefix. /// - /// Given a table URI, object URI can be constructed by joining table URI with object path. - pub path: String, - /// The last time the object was modified in the storage backend. - // The timestamp of a commit comes from the remote storage `lastModifiedTime`, and can be - // adjusted for clock skew. - pub modified: DateTime, - /// Size of the object in bytes - pub size: Option, -} - -impl Clone for ObjectMeta { - fn clone(&self) -> Self { - Self { - path: self.path.clone(), - modified: self.modified, - size: self.size, - } + /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of + /// `foo/bar_baz/x`. + async fn list( + &self, + prefix: Option<&Path>, + ) -> ObjectStoreResult>> { + let prefix = prefix.and_then(|p| self.config.full_path(p).ok()); + Ok(self + .storage + .list(Some( + &prefix.unwrap_or_else(|| self.config.storage_url.prefix.clone()), + )) + .await? + .map_ok(|meta| ObjectMeta { + last_modified: meta.last_modified, + size: meta.size, + location: self + .config + .strip_prefix(&meta.location) + .unwrap_or(meta.location), + }) + .boxed()) } -} - -/// Abstractions for underlying blob storages hosting the Delta table. To add support for new cloud -/// or local storage systems, simply implement this trait. -#[async_trait::async_trait] -pub trait StorageBackend: Send + Sync + Debug { - /// Fetch object metadata without reading the actual content - async fn head_obj(&self, path: &str) -> Result; - - /// Fetch object content - async fn get_obj(&self, path: &str) -> Result, StorageError>; - - /// Fetch a range from object content - async fn get_range(&self, path: &str, range: Range) -> Result, StorageError>; - /// Return a list of objects by `path` prefix in an async stream. - async fn list_objs<'a>( - &'a self, - path: &'a str, - ) -> Result>, StorageError>; - - /// Create new object with `obj_bytes` as content. - /// - /// Implementation note: + /// List objects with the given prefix and an implementation specific + /// delimiter. Returns common prefixes (directories) in addition to object + /// metadata. /// - /// To support safe concurrent read, if `path` already exists, `put_obj` needs to update object - /// content in backing store atomically, i.e. reader of the object should never read a partial - /// write. - async fn put_obj(&self, path: &str, obj_bytes: &[u8]) -> Result<(), StorageError>; + /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of + /// `foo/bar_baz/x`. + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { + let prefix = prefix.and_then(|p| self.config.full_path(p).ok()); + self.storage + .list_with_delimiter(Some( + &prefix.unwrap_or_else(|| self.config.storage_url.prefix.clone()), + )) + .await + .map(|lst| ListResult { + common_prefixes: lst + .common_prefixes + .iter() + .map(|p| self.config.strip_prefix(p).unwrap_or_else(|| p.clone())) + .collect(), + objects: lst + .objects + .iter() + .map(|meta| ObjectMeta { + last_modified: meta.last_modified, + size: meta.size, + location: self + .config + .strip_prefix(&meta.location) + .unwrap_or_else(|| meta.location.clone()), + }) + .collect(), + }) + } - /// Moves object from `src` to `dst`. + /// Copy an object from one path to another in the same object store. /// - /// Implementation note: - /// - /// For a multi-writer safe backend, `rename_obj_noreplace` needs to implement rename if not exists semantic. - /// In other words, if the destination path already exists, rename should return a - /// [StorageError::AlreadyExists] error. - async fn rename_obj_noreplace(&self, src: &str, dst: &str) -> Result<(), StorageError>; + /// If there exists an object at the destination, it will be overwritten. + async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + let full_from = self.config.full_path(from)?; + let full_to = self.config.full_path(to)?; + self.storage.copy(&full_from, &full_to).await + } - /// Deletes object by `path`. - async fn delete_obj(&self, path: &str) -> Result<(), StorageError>; + /// Copy an object from one path to another, only if destination is empty. + /// + /// Will return an error if the destination already has an object. + async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + let full_from = self.config.full_path(from)?; + let full_to = self.config.full_path(to)?; + self.storage.copy_if_not_exists(&full_from, &full_to).await + } - /// Deletes object by `paths`. - async fn delete_objs(&self, paths: &[String]) -> Result<(), StorageError> { - for path in paths { - match self.delete_obj(path).await { - Ok(_) => continue, - Err(StorageError::NotFound) => continue, - Err(e) => return Err(e), - } - } - Ok(()) + /// Move an object from one path to another in the same object store. + /// + /// Will return an error if the destination already has an object. + async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + let full_from = self.config.full_path(from)?; + let full_to = self.config.full_path(to)?; + self.storage + .rename_if_not_exists(&full_from, &full_to) + .await } -} -/// Dynamically construct a Storage backend trait object based on scheme for provided URI -pub fn get_backend_for_uri(uri: &str) -> Result, StorageError> { - match parse_uri(uri)? { - Uri::LocalPath(root) => Ok(Arc::new(file::FileStorageBackend::new(root))), - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - Uri::S3Object(_) => Ok(Arc::new(s3::S3StorageBackend::new()?)), - #[cfg(feature = "azure")] - Uri::AdlsGen2Object(obj) => Ok(Arc::new(azure::AdlsGen2Backend::new(obj.file_system)?)), - #[cfg(feature = "gcs")] - Uri::GCSObject(_) => Ok(Arc::new(gcs::GCSStorageBackend::new()?)), + async fn put_multipart( + &self, + location: &Path, + ) -> ObjectStoreResult<(MultipartId, Box)> { + let full_path = self.config.full_path(location)?; + self.storage.put_multipart(&full_path).await } -} -/// Returns a StorageBackend appropriate for the protocol and configured with the given options -/// Options must be passed as a hashmap. Hashmap keys correspond to env variables that are used if options are not set. -/// -/// Currently, S3 and Azure are the only backends that accept options. -/// Options may be passed in the HashMap or set as environment variables. -/// -/// [s3::S3StorageOptions] describes the available options for the S3 backend. -/// [dynamodb_lock::DynamoDbLockClient] describes additional options for the atomic rename client. -/// -/// [azure::AzureStorageOptions] describes the available options for the Azure backend. -pub fn get_backend_for_uri_with_options( - uri: &str, - #[allow(unused)] options: HashMap, -) -> Result, StorageError> { - match parse_uri(uri)? { - #[cfg(any(feature = "s3", feature = "s3-rustls"))] - Uri::S3Object(_) => Ok(Arc::new(s3::S3StorageBackend::new_from_options( - s3::S3StorageOptions::from_map(options), - )?)), - #[cfg(feature = "azure")] - Uri::AdlsGen2Object(obj) => Ok(Arc::new(azure::AdlsGen2Backend::new_from_options( - obj.file_system, - azure::AzureStorageOptions::from_map(options), - )?)), - _ => get_backend_for_uri(uri), + async fn abort_multipart( + &self, + location: &Path, + multipart_id: &MultipartId, + ) -> ObjectStoreResult<()> { + let full_path = self.config.full_path(location)?; + self.storage.abort_multipart(&full_path, multipart_id).await } } -#[cfg(any(feature = "s3", feature = "s3-rustls", feature = "azure"))] -pub(crate) fn str_option(map: &HashMap, key: &str) -> Option { - map.get(key) - .map_or_else(|| std::env::var(key).ok(), |v| Some(v.to_owned())) +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn path_handling() { + let storage_url = StorageUrl::parse("s3://bucket").unwrap(); + let file_with_delimiter = Path::from_iter(["a", "b/c", "foo.file"]); + let config = DeltaObjectStoreConfig::new(storage_url); + let added = config.full_path(&file_with_delimiter).unwrap(); + assert_eq!(file_with_delimiter, added) + } } diff --git a/rust/src/storage/s3.rs b/rust/src/storage/s3.rs new file mode 100644 index 0000000000..483f215704 --- /dev/null +++ b/rust/src/storage/s3.rs @@ -0,0 +1,702 @@ +//! AWS S3 storage backend. + +use crate::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; +use object_store::path::Path; +use object_store::{ + DynObjectStore, Error as ObjectStoreError, GetResult, ListResult, MultipartId, ObjectMeta, + ObjectStore, Result as ObjectStoreResult, +}; +use rusoto_core::{HttpClient, Region}; +use rusoto_credential::AutoRefreshingProvider; +use rusoto_sts::WebIdentityProvider; +use serde::Deserialize; +use serde::Serialize; +use std::collections::HashMap; +use std::fmt::Debug; +use std::ops::Range; +use std::sync::Arc; +use std::time::Duration; +use tokio::io::AsyncWrite; + +const STORE_NAME: &str = "DeltaS3ObjectStore"; + +/// Error raised by storage lock client +#[derive(thiserror::Error, Debug)] +enum S3LockError { + /// Error raised when (de)serializing data. + #[error("Error serializing lock data: {source}")] + Serde { + /// raw error + source: serde_json::Error, + }, + + /// Error raised for failed lock acquisition + #[error("Failed acquiring lock after {attempts} attempts.")] + AcquireLock { + /// number of attempts + attempts: u32, + }, + + /// Error raised for failed lock release + #[error("Failed releasing lock for item: {:?}", item)] + ReleaseLock { + /// related lock item + item: LockItem, + }, + + /// Error interacting with dynamo lock client + #[error("Dynamo Error: {} ({:?}).", source, source)] + Dynamo { + /// raw error + source: DynamoError, + }, + + /// Error raised when required lock data si missing + #[error("Missing lock data for item: {:?}.", item)] + MissingData { + /// related lock item + item: LockItem, + }, + + /// Error raised getting credentials + #[error("Failed to retrieve AWS credentials: {source}")] + Credentials { + /// The underlying Rusoto CredentialsError + #[from] + source: rusoto_credential::CredentialsError, + }, + + /// Error raised creating http client + #[error("Failed to create request dispatcher: {source}")] + HttpClient { + /// The underlying Rusoto TlsError + #[from] + source: rusoto_core::request::TlsError, + }, + + #[error("Rename target already exists")] + AlreadyExists, + + #[error("Atomic rename requires a LockClient for S3 backends.")] + LockClientRequired, +} + +impl From for ObjectStoreError { + fn from(e: S3LockError) -> Self { + ObjectStoreError::Generic { + store: STORE_NAME, + source: Box::new(e), + } + } +} + +/// Lock data which stores an attempt to rename `source` into `destination` +#[derive(Clone, Debug, Serialize, Deserialize)] +pub struct LockData { + /// Source object key + pub source: String, + /// Destination object ket + pub destination: String, +} + +impl LockData { + /// Builds new `LockData` instance and then creates json string from it. + pub fn json(src: &str, dst: &str) -> Result { + let data = LockData { + source: src.to_string(), + destination: dst.to_string(), + }; + serde_json::to_string(&data) + } +} + +/// Uses a `LockClient` to support additional features required by S3 Storage. +struct S3LockClient { + lock_client: Box, +} + +impl S3LockClient { + async fn rename_with_lock( + &self, + s3: &S3StorageBackend, + src: &Path, + dst: &Path, + ) -> Result<(), ObjectStoreError> { + let mut lock = self.acquire_lock_loop(src.as_ref(), dst.as_ref()).await?; + + if let Some(ref data) = lock.data { + let data: LockData = + serde_json::from_str(data).map_err(|err| S3LockError::Serde { source: err })?; + + if lock.acquired_expired_lock { + log::info!( + "Acquired expired lock. Repairing the rename of {} to {}.", + &data.source, + &data.destination + ); + } + + let mut rename_result = s3 + .rename_no_replace(&Path::from(data.source), &Path::from(data.destination)) + .await; + + if lock.acquired_expired_lock { + match rename_result { + // AlreadyExists when the stale rename is done, but the lock not released + // NotFound when the source file of rename is missing + Err(ObjectStoreError::AlreadyExists { .. }) + | Err(ObjectStoreError::NotFound { .. }) => (), + _ => rename_result?, + } + + // If we acquired expired lock then the rename done above is + // a repair of expired one. So on this time we try the intended rename. + lock.data = Some( + LockData::json(src.as_ref(), dst.as_ref()) + .map_err(|err| S3LockError::Serde { source: err })?, + ); + lock = self + .lock_client + .update_data(&lock) + .await + .map_err(|err| S3LockError::Dynamo { source: err })?; + rename_result = s3.rename_no_replace(src, dst).await; + } + + let release_result = self.lock_client.release_lock(&lock).await; + + // before unwrapping `rename_result` the `release_result` is called + // to ensure that we no longer hold the lock + rename_result?; + + if !release_result.map_err(|err| S3LockError::Dynamo { source: err })? { + return Err(S3LockError::ReleaseLock { item: lock }.into()); + } + + Ok(()) + } else { + Err(S3LockError::MissingData { item: lock }.into()) + } + } + + async fn acquire_lock_loop(&self, src: &str, dst: &str) -> Result { + let data = LockData::json(src, dst).map_err(|err| S3LockError::Serde { source: err })?; + + let lock; + let mut retries = 0; + + loop { + match self + .lock_client + .try_acquire_lock(data.as_str()) + .await + .map_err(|err| S3LockError::Dynamo { source: err })? + { + Some(l) => { + lock = l; + break; + } + None => { + retries += 1; + if retries > DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS { + return Err(S3LockError::AcquireLock { + attempts: DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS, + }); + } + } + } + } + + Ok(lock) + } +} + +/// Options used to configure the S3StorageBackend. +/// +/// Available options are described in [s3_storage_options]. +#[derive(Clone, Debug, PartialEq, Eq)] +#[allow(missing_docs)] +pub struct S3StorageOptions { + pub endpoint_url: Option, + pub region: Region, + pub aws_access_key_id: Option, + pub aws_secret_access_key: Option, + pub aws_session_token: Option, + pub locking_provider: Option, + pub assume_role_arn: Option, + pub assume_role_session_name: Option, + pub use_web_identity: bool, + pub s3_pool_idle_timeout: Duration, + pub sts_pool_idle_timeout: Duration, + pub s3_get_internal_server_error_retries: usize, + pub extra_opts: HashMap, +} + +impl S3StorageOptions { + /// Creates an instance of S3StorageOptions from the given HashMap. + pub fn from_map(options: HashMap) -> S3StorageOptions { + let extra_opts = options + .iter() + .filter(|(k, _)| !s3_storage_options::S3_OPTS.contains(&k.as_str())) + .map(|(k, v)| (k.to_owned(), v.to_owned())) + .collect(); + + // Copy web identity values provided in options but not the environment into the environment + // to get picked up by the `from_k8s_env` call in `get_web_identity_provider`. + Self::ensure_env_var(&options, s3_storage_options::AWS_REGION); + Self::ensure_env_var(&options, s3_storage_options::AWS_ACCESS_KEY_ID); + Self::ensure_env_var(&options, s3_storage_options::AWS_SECRET_ACCESS_KEY); + Self::ensure_env_var(&options, s3_storage_options::AWS_SESSION_TOKEN); + Self::ensure_env_var(&options, s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE); + Self::ensure_env_var(&options, s3_storage_options::AWS_ROLE_ARN); + Self::ensure_env_var(&options, s3_storage_options::AWS_ROLE_SESSION_NAME); + + let endpoint_url = str_option(&options, s3_storage_options::AWS_ENDPOINT_URL); + let region = if let Some(endpoint_url) = endpoint_url.as_ref() { + Region::Custom { + name: Self::str_or_default( + &options, + s3_storage_options::AWS_REGION, + "custom".to_string(), + ), + endpoint: endpoint_url.to_owned(), + } + } else { + Region::default() + }; + + let s3_pool_idle_timeout = Self::u64_or_default( + &options, + s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, + 15, + ); + let sts_pool_idle_timeout = Self::u64_or_default( + &options, + s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, + 10, + ); + + let s3_get_internal_server_error_retries = Self::u64_or_default( + &options, + s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + 10, + ) as usize; + + Self { + endpoint_url, + region, + aws_access_key_id: str_option(&options, s3_storage_options::AWS_ACCESS_KEY_ID), + aws_secret_access_key: str_option(&options, s3_storage_options::AWS_SECRET_ACCESS_KEY), + aws_session_token: str_option(&options, s3_storage_options::AWS_SESSION_TOKEN), + locking_provider: str_option(&options, s3_storage_options::AWS_S3_LOCKING_PROVIDER), + assume_role_arn: str_option(&options, s3_storage_options::AWS_S3_ASSUME_ROLE_ARN), + assume_role_session_name: str_option( + &options, + s3_storage_options::AWS_S3_ROLE_SESSION_NAME, + ), + use_web_identity: std::env::var(s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE) + .is_ok(), + s3_pool_idle_timeout: Duration::from_secs(s3_pool_idle_timeout), + sts_pool_idle_timeout: Duration::from_secs(sts_pool_idle_timeout), + s3_get_internal_server_error_retries, + extra_opts, + } + } + + fn str_or_default(map: &HashMap, key: &str, default: String) -> String { + map.get(key) + .map(|v| v.to_owned()) + .unwrap_or_else(|| std::env::var(key).unwrap_or(default)) + } + + fn u64_or_default(map: &HashMap, key: &str, default: u64) -> u64 { + str_option(map, key) + .and_then(|v| v.parse().ok()) + .unwrap_or(default) + } + + fn ensure_env_var(map: &HashMap, key: &str) { + if let Some(val) = str_option(map, key) { + std::env::set_var(key, val); + } + } +} + +impl Default for S3StorageOptions { + /// Creates an instance of S3StorageOptions from environment variables. + fn default() -> S3StorageOptions { + Self::from_map(HashMap::new()) + } +} + +fn get_web_identity_provider() -> Result, S3LockError> { + let provider = WebIdentityProvider::from_k8s_env(); + Ok(AutoRefreshingProvider::new(provider)?) +} + +/// An S3 implementation of the [ObjectStore] trait +/// +/// The backend can optionally use [dynamodb_lock] to better support concurrent writers. +pub struct S3StorageBackend { + inner: Arc, + s3_lock_client: Option, +} + +impl std::fmt::Display for S3StorageBackend { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "S3StorageBackend") + } +} + +impl S3StorageBackend { + /// Creates a new S3StorageBackend, trying to create lock client from options. + /// + /// Options are described in [s3_storage_options]. + /// + /// ```rust + /// use object_store::aws::AmazonS3Builder; + /// use deltalake::storage::s3::{S3StorageBackend, S3StorageOptions}; + /// use std::sync::Arc; + /// + /// let inner = AmazonS3Builder::new() + /// .with_region("us-east-1") + /// .with_bucket_name("my-bucket") + /// .with_access_key_id("") + /// .with_secret_access_key("") + /// .build() + /// .unwrap(); + /// let store = S3StorageBackend::try_new(Arc::new(inner), S3StorageOptions::default()).unwrap(); + /// ``` + pub fn try_new( + storage: Arc, + options: S3StorageOptions, + ) -> ObjectStoreResult { + let s3_lock_client = try_create_lock_client(&options)?; + Ok(Self { + inner: storage, + s3_lock_client, + }) + } + + /// Creates a new S3StorageBackend with given options, s3 client and lock client. + pub fn with_lock_client( + storage: Arc, + lock_client: Option>, + ) -> Self { + let s3_lock_client = lock_client.map(|lc| S3LockClient { lock_client: lc }); + Self { + inner: storage, + s3_lock_client, + } + } + + pub(self) async fn rename_no_replace(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + match self.head(to).await { + Ok(_) => { + return Err(ObjectStoreError::AlreadyExists { + path: to.to_string(), + source: Box::new(S3LockError::AlreadyExists), + }) + } + Err(ObjectStoreError::NotFound { .. }) => (), + Err(e) => return Err(e), + } + self.inner.rename(from, to).await + } +} + +impl std::fmt::Debug for S3StorageBackend { + fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(fmt, "S3StorageBackend") + } +} + +#[async_trait::async_trait] +impl ObjectStore for S3StorageBackend { + async fn put(&self, location: &Path, bytes: Bytes) -> ObjectStoreResult<()> { + self.inner.put(location, bytes).await + } + + async fn get(&self, location: &Path) -> ObjectStoreResult { + self.inner.get(location).await + } + + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + self.inner.get_range(location, range).await + } + + async fn head(&self, location: &Path) -> ObjectStoreResult { + self.inner.head(location).await + } + + async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { + self.inner.delete(location).await + } + + async fn list( + &self, + prefix: Option<&Path>, + ) -> ObjectStoreResult>> { + self.inner.list(prefix).await + } + + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { + self.inner.list_with_delimiter(prefix).await + } + + async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + self.inner.copy(from, to).await + } + + async fn copy_if_not_exists(&self, _from: &Path, _to: &Path) -> ObjectStoreResult<()> { + todo!() + } + + async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + let lock_client = match self.s3_lock_client { + Some(ref lock_client) => lock_client, + None => return Err(S3LockError::LockClientRequired.into()), + }; + lock_client.rename_with_lock(self, from, to).await?; + Ok(()) + } + + async fn put_multipart( + &self, + location: &Path, + ) -> ObjectStoreResult<(MultipartId, Box)> { + self.inner.put_multipart(location).await + } + + async fn abort_multipart( + &self, + location: &Path, + multipart_id: &MultipartId, + ) -> ObjectStoreResult<()> { + self.inner.abort_multipart(location, multipart_id).await + } +} + +fn try_create_lock_client(options: &S3StorageOptions) -> Result, S3LockError> { + let dispatcher = HttpClient::new()?; + + match &options.locking_provider { + Some(p) if p.to_lowercase() == "dynamodb" => { + let dynamodb_client = match options.use_web_identity { + true => rusoto_dynamodb::DynamoDbClient::new_with( + dispatcher, + get_web_identity_provider()?, + options.region.clone(), + ), + false => rusoto_dynamodb::DynamoDbClient::new(options.region.clone()), + }; + let lock_client = dynamodb_lock::DynamoDbLockClient::new( + dynamodb_client, + dynamodb_lock::DynamoDbOptions::from_map(options.extra_opts.clone()), + ); + Ok(Some(S3LockClient { + lock_client: Box::new(lock_client), + })) + } + _ => Ok(None), + } +} + +#[cfg(test)] +#[cfg(not(feature = "integration_test"))] +mod tests { + use super::*; + + use maplit::hashmap; + use serial_test::serial; + + #[test] + #[serial] + fn storage_options_default_test() { + std::env::set_var(s3_storage_options::AWS_ENDPOINT_URL, "http://localhost"); + std::env::set_var(s3_storage_options::AWS_REGION, "us-west-1"); + std::env::set_var(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + std::env::set_var( + s3_storage_options::AWS_S3_ASSUME_ROLE_ARN, + "arn:aws:iam::123456789012:role/some_role", + ); + std::env::set_var(s3_storage_options::AWS_S3_ROLE_SESSION_NAME, "session_name"); + std::env::set_var( + s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE, + "token_file", + ); + std::env::remove_var(s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS); + std::env::remove_var(s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS); + std::env::remove_var(s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES); + + let options = S3StorageOptions::default(); + + assert_eq!( + S3StorageOptions { + endpoint_url: Some("http://localhost".to_string()), + region: Region::Custom { + name: "us-west-1".to_string(), + endpoint: "http://localhost".to_string() + }, + aws_access_key_id: Some("test".to_string()), + aws_secret_access_key: Some("test".to_string()), + aws_session_token: None, + assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), + assume_role_session_name: Some("session_name".to_string()), + use_web_identity: true, + locking_provider: Some("dynamodb".to_string()), + s3_pool_idle_timeout: Duration::from_secs(15), + sts_pool_idle_timeout: Duration::from_secs(10), + s3_get_internal_server_error_retries: 10, + extra_opts: HashMap::new(), + }, + options + ); + } + + #[test] + #[serial] + fn storage_options_with_only_region_and_credentials() { + std::env::remove_var(s3_storage_options::AWS_ENDPOINT_URL); + let options = S3StorageOptions::from_map(hashmap! { + s3_storage_options::AWS_REGION.to_string() => "eu-west-1".to_string(), + s3_storage_options::AWS_ACCESS_KEY_ID.to_string() => "test".to_string(), + s3_storage_options::AWS_SECRET_ACCESS_KEY.to_string() => "test".to_string(), + }); + + assert_eq!( + S3StorageOptions { + endpoint_url: None, + region: Region::default(), + aws_access_key_id: Some("test".to_string()), + aws_secret_access_key: Some("test".to_string()), + ..Default::default() + }, + options + ); + } + + #[test] + #[serial] + fn storage_options_from_map_test() { + let options = S3StorageOptions::from_map(hashmap! { + s3_storage_options::AWS_ENDPOINT_URL.to_string() => "http://localhost:1234".to_string(), + s3_storage_options::AWS_REGION.to_string() => "us-west-2".to_string(), + s3_storage_options::AWS_S3_LOCKING_PROVIDER.to_string() => "another_locking_provider".to_string(), + s3_storage_options::AWS_S3_ASSUME_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/another_role".to_string(), + s3_storage_options::AWS_S3_ROLE_SESSION_NAME.to_string() => "another_session_name".to_string(), + s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "another_token_file".to_string(), + s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "1".to_string(), + s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "2".to_string(), + s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES.to_string() => "3".to_string(), + }); + + assert_eq!( + S3StorageOptions { + endpoint_url: Some("http://localhost:1234".to_string()), + region: Region::Custom { + name: "us-west-2".to_string(), + endpoint: "http://localhost:1234".to_string() + }, + aws_access_key_id: Some("test".to_string()), + aws_secret_access_key: Some("test".to_string()), + aws_session_token: None, + assume_role_arn: Some("arn:aws:iam::123456789012:role/another_role".to_string()), + assume_role_session_name: Some("another_session_name".to_string()), + use_web_identity: true, + locking_provider: Some("another_locking_provider".to_string()), + s3_pool_idle_timeout: Duration::from_secs(1), + sts_pool_idle_timeout: Duration::from_secs(2), + s3_get_internal_server_error_retries: 3, + extra_opts: HashMap::new(), + }, + options + ); + } + + #[test] + #[serial] + fn storage_options_mixed_test() { + std::env::set_var(s3_storage_options::AWS_ENDPOINT_URL, "http://localhost"); + std::env::set_var(s3_storage_options::AWS_REGION, "us-west-1"); + std::env::set_var(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + std::env::set_var( + s3_storage_options::AWS_S3_ASSUME_ROLE_ARN, + "arn:aws:iam::123456789012:role/some_role", + ); + std::env::set_var(s3_storage_options::AWS_S3_ROLE_SESSION_NAME, "session_name"); + std::env::set_var( + s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE, + "token_file", + ); + + std::env::set_var(s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, "1"); + std::env::set_var(s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, "2"); + std::env::set_var( + s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, + "3", + ); + let options = S3StorageOptions::from_map(hashmap! { + s3_storage_options::AWS_REGION.to_string() => "us-west-2".to_string(), + "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), + "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES".to_string() => "3".to_string(), + }); + + assert_eq!( + S3StorageOptions { + endpoint_url: Some("http://localhost".to_string()), + region: Region::Custom { + name: "us-west-2".to_string(), + endpoint: "http://localhost".to_string() + }, + aws_access_key_id: Some("test".to_string()), + aws_secret_access_key: Some("test".to_string()), + aws_session_token: None, + assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), + assume_role_session_name: Some("session_name".to_string()), + use_web_identity: true, + locking_provider: Some("dynamodb".to_string()), + s3_pool_idle_timeout: Duration::from_secs(1), + sts_pool_idle_timeout: Duration::from_secs(2), + s3_get_internal_server_error_retries: 3, + extra_opts: hashmap! { + "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), + }, + }, + options + ); + } + #[test] + #[serial] + fn storage_options_web_identity_test() { + let _options = S3StorageOptions::from_map(hashmap! { + s3_storage_options::AWS_REGION.to_string() => "eu-west-1".to_string(), + s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "web_identity_token_file".to_string(), + s3_storage_options::AWS_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/web_identity_role".to_string(), + s3_storage_options::AWS_ROLE_SESSION_NAME.to_string() => "web_identity_session_name".to_string(), + }); + + assert_eq!( + "eu-west-1", + std::env::var(s3_storage_options::AWS_REGION).unwrap() + ); + + assert_eq!( + "web_identity_token_file", + std::env::var(s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE).unwrap() + ); + + assert_eq!( + "arn:aws:iam::123456789012:role/web_identity_role", + std::env::var(s3_storage_options::AWS_ROLE_ARN).unwrap() + ); + + assert_eq!( + "web_identity_session_name", + std::env::var(s3_storage_options::AWS_ROLE_SESSION_NAME).unwrap() + ); + } +} diff --git a/rust/src/storage/s3/mod.rs b/rust/src/storage/s3/mod.rs deleted file mode 100644 index 7c1b669454..0000000000 --- a/rust/src/storage/s3/mod.rs +++ /dev/null @@ -1,1178 +0,0 @@ -//! AWS S3 storage backend. It only supports a single writer and is not multi-writer safe. - -use std::collections::HashMap; -use std::fmt; -use std::fmt::Debug; -use std::ops::Range; - -use chrono::{DateTime, FixedOffset, Utc}; -use futures::stream::BoxStream; - -use log::debug; -use rusoto_core::{HttpClient, HttpConfig, Region, RusotoError}; -use rusoto_credential::AutoRefreshingProvider; -use rusoto_s3::{ - CopyObjectRequest, Delete, DeleteObjectRequest, DeleteObjectsRequest, GetObjectError, - GetObjectOutput, GetObjectRequest, HeadObjectRequest, ListObjectsV2Request, ObjectIdentifier, - PutObjectRequest, S3Client, S3, -}; -use rusoto_sts::{StsAssumeRoleSessionCredentialsProvider, StsClient, WebIdentityProvider}; -use tokio::io::AsyncReadExt; - -use super::{parse_uri, str_option, ObjectMeta, StorageBackend, StorageError}; -use rusoto_core::credential::{ - AwsCredentials, CredentialsError, DefaultCredentialsProvider, ProvideAwsCredentials, -}; -use serde::Deserialize; -use serde::Serialize; -use std::time::Duration; -use uuid::Uuid; - -use dynamodb_lock::{LockClient, LockItem, DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS}; - -use hyper::client::HttpConnector; -use hyper_proxy::{Intercept, Proxy, ProxyConnector}; -use hyper_rustls::{HttpsConnector, HttpsConnectorBuilder}; - -use std::env; - -/// Lock data which stores an attempt to rename `source` into `destination` -#[derive(Clone, Debug, Serialize, Deserialize)] -pub struct LockData { - /// Source object key - pub source: String, - /// Destination object ket - pub destination: String, -} - -impl LockData { - /// Builds new `LockData` instance and then creates json string from it. - pub fn json(src: &str, dst: &str) -> Result { - let data = LockData { - source: src.to_string(), - destination: dst.to_string(), - }; - let json = serde_json::to_string(&data) - .map_err(|_| StorageError::S3Generic("Lock data serialize error".to_string()))?; - - Ok(json) - } -} - -/// Uses a `LockClient` to support additional features required by S3 Storage. -pub struct S3LockClient { - lock_client: Box, -} - -impl S3LockClient { - async fn rename_with_lock( - &self, - s3: &S3StorageBackend, - src: &str, - dst: &str, - ) -> Result<(), StorageError> { - let mut lock = self.acquire_lock_loop(src, dst).await?; - - if let Some(ref data) = lock.data { - let data: LockData = serde_json::from_str(data) - .map_err(|_| StorageError::S3Generic("Lock data deserialize error".to_string()))?; - - if lock.acquired_expired_lock { - log::info!( - "Acquired expired lock. Repairing the rename of {} to {}.", - &data.source, - &data.destination - ); - } - - let mut rename_result = s3.unsafe_rename_obj(&data.source, &data.destination).await; - - if lock.acquired_expired_lock { - match rename_result { - // AlreadyExists when the stale rename is done, but the lock not released - // NotFound when the source file of rename is missing - Err(StorageError::AlreadyExists(_)) | Err(StorageError::NotFound) => (), - _ => rename_result?, - } - - // If we acquired expired lock then the rename done above is - // a repair of expired one. So on this time we try the intended rename. - lock.data = Some(LockData::json(src, dst)?); - lock = self.lock_client.update_data(&lock).await?; - rename_result = s3.unsafe_rename_obj(src, dst).await; - } - - let release_result = self.lock_client.release_lock(&lock).await; - - // before unwrapping `rename_result` the `release_result` is called to ensure that we - // no longer hold the lock - rename_result?; - - if !release_result? { - log::error!("Could not release lock {:?}", &lock); - return Err(StorageError::S3Generic("Lock is not released".to_string())); - } - - Ok(()) - } else { - Err(StorageError::S3Generic( - "Acquired lock with no lock data".to_string(), - )) - } - } - - async fn acquire_lock_loop(&self, src: &str, dst: &str) -> Result { - let data = LockData::json(src, dst)?; - - let lock; - let mut retries = 0; - - loop { - match self.lock_client.try_acquire_lock(data.as_str()).await? { - Some(l) => { - lock = l; - break; - } - None => { - retries += 1; - if retries > DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS { - return Err(StorageError::S3Generic("Cannot acquire lock".to_string())); - } - } - } - } - - Ok(lock) - } -} - -/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. -/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. -/// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. -pub mod s3_storage_options { - /// Custom S3 endpoint. - pub const AWS_ENDPOINT_URL: &str = "AWS_ENDPOINT_URL"; - /// The AWS region. - pub const AWS_REGION: &str = "AWS_REGION"; - /// The AWS_ACCESS_KEY_ID to use for S3. - pub const AWS_ACCESS_KEY_ID: &str = "AWS_ACCESS_KEY_ID"; - /// The AWS_SECRET_ACCESS_ID to use for S3. - pub const AWS_SECRET_ACCESS_KEY: &str = "AWS_SECRET_ACCESS_KEY"; - /// The AWS_SESSION_TOKEN to use for S3. - pub const AWS_SESSION_TOKEN: &str = "AWS_SESSION_TOKEN"; - /// Locking provider to use for safe atomic rename. - /// `dynamodb` is currently the only supported locking provider. - /// If not set, safe atomic rename is not available. - pub const AWS_S3_LOCKING_PROVIDER: &str = "AWS_S3_LOCKING_PROVIDER"; - /// The role to assume for S3 writes. - pub const AWS_S3_ASSUME_ROLE_ARN: &str = "AWS_S3_ASSUME_ROLE_ARN"; - /// The role session name to use when a role is assumed. If not provided a random session name is generated. - pub const AWS_S3_ROLE_SESSION_NAME: &str = "AWS_S3_ROLE_SESSION_NAME"; - /// The `pool_idle_timeout` option of aws http client. Has to be lower than 20 seconds, which is - /// default S3 server timeout . - /// However, since rusoto uses hyper as a client, its default timeout is 90 seconds - /// . - /// Hence, the `connection closed before message completed` could occur. - /// To avoid that, the default value of this setting is 15 seconds if it's not set otherwise. - pub const AWS_S3_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_S3_POOL_IDLE_TIMEOUT_SECONDS"; - /// The `pool_idle_timeout` for the as3_storage_optionsws sts client. See - /// the reasoning in `AWS_S3_POOL_IDLE_TIMEOUT_SECONDS`. - pub const AWS_STS_POOL_IDLE_TIMEOUT_SECONDS: &str = "AWS_STS_POOL_IDLE_TIMEOUT_SECONDS"; - /// The number of retries for S3 GET requests failed with 500 Internal Server Error. - pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = - "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES"; - /// The web identity token file to use when using a web identity provider. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; - /// The role name to use for web identity. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; - /// The role session name to use for web identity. - /// NOTE: web identity related options are set in the environment when - /// creating an instance of [crate::storage::s3::S3StorageOptions]. - /// See also . - pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; - - /// The list of option keys owned by the S3 module. - /// Option keys not contained in this list will be added to the `extra_opts` - /// field of [crate::storage::s3::S3StorageOptions]. - /// `extra_opts` are passed to [dynamodb_lock::DynamoDbOptions] to configure the lock client. - pub const S3_OPTS: &[&str] = &[ - AWS_ENDPOINT_URL, - AWS_REGION, - AWS_ACCESS_KEY_ID, - AWS_SECRET_ACCESS_KEY, - AWS_SESSION_TOKEN, - AWS_S3_LOCKING_PROVIDER, - AWS_S3_ASSUME_ROLE_ARN, - AWS_S3_ROLE_SESSION_NAME, - AWS_WEB_IDENTITY_TOKEN_FILE, - AWS_ROLE_ARN, - AWS_ROLE_SESSION_NAME, - AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, - AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, - AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - ]; -} - -/// Options used to configure the S3StorageBackend. -/// -/// Available options are described in [s3_storage_options]. -#[derive(Clone, Debug, PartialEq, Eq)] -pub struct S3StorageOptions { - _endpoint_url: Option, - region: Region, - aws_access_key_id: Option, - aws_secret_access_key: Option, - aws_session_token: Option, - locking_provider: Option, - assume_role_arn: Option, - assume_role_session_name: Option, - use_web_identity: bool, - s3_pool_idle_timeout: Duration, - sts_pool_idle_timeout: Duration, - s3_get_internal_server_error_retries: usize, - extra_opts: HashMap, -} - -impl S3StorageOptions { - /// Creates an instance of S3StorageOptions from the given HashMap. - pub fn from_map(options: HashMap) -> S3StorageOptions { - let extra_opts = options - .iter() - .filter(|(k, _)| !s3_storage_options::S3_OPTS.contains(&k.as_str())) - .map(|(k, v)| (k.to_owned(), v.to_owned())) - .collect(); - - // Copy web identity values provided in options but not the environment into the environment - // to get picked up by the `from_k8s_env` call in `get_web_identity_provider`. - Self::ensure_env_var(&options, s3_storage_options::AWS_REGION); - Self::ensure_env_var(&options, s3_storage_options::AWS_ACCESS_KEY_ID); - Self::ensure_env_var(&options, s3_storage_options::AWS_SECRET_ACCESS_KEY); - Self::ensure_env_var(&options, s3_storage_options::AWS_SESSION_TOKEN); - Self::ensure_env_var(&options, s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE); - Self::ensure_env_var(&options, s3_storage_options::AWS_ROLE_ARN); - Self::ensure_env_var(&options, s3_storage_options::AWS_ROLE_SESSION_NAME); - - let endpoint_url = str_option(&options, s3_storage_options::AWS_ENDPOINT_URL); - let region = if let Some(endpoint_url) = endpoint_url.as_ref() { - Region::Custom { - name: Self::str_or_default( - &options, - s3_storage_options::AWS_REGION, - "custom".to_string(), - ), - endpoint: endpoint_url.to_owned(), - } - } else { - Region::default() - }; - - let s3_pool_idle_timeout = Self::u64_or_default( - &options, - s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, - 15, - ); - let sts_pool_idle_timeout = Self::u64_or_default( - &options, - s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, - 10, - ); - - let s3_get_internal_server_error_retries = Self::u64_or_default( - &options, - s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - 10, - ) as usize; - - Self { - _endpoint_url: endpoint_url, - region, - aws_access_key_id: str_option(&options, s3_storage_options::AWS_ACCESS_KEY_ID), - aws_secret_access_key: str_option(&options, s3_storage_options::AWS_SECRET_ACCESS_KEY), - aws_session_token: str_option(&options, s3_storage_options::AWS_SESSION_TOKEN), - locking_provider: str_option(&options, s3_storage_options::AWS_S3_LOCKING_PROVIDER), - assume_role_arn: str_option(&options, s3_storage_options::AWS_S3_ASSUME_ROLE_ARN), - assume_role_session_name: str_option( - &options, - s3_storage_options::AWS_S3_ROLE_SESSION_NAME, - ), - use_web_identity: std::env::var(s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE) - .is_ok(), - s3_pool_idle_timeout: Duration::from_secs(s3_pool_idle_timeout), - sts_pool_idle_timeout: Duration::from_secs(sts_pool_idle_timeout), - s3_get_internal_server_error_retries, - extra_opts, - } - } - - fn str_or_default(map: &HashMap, key: &str, default: String) -> String { - map.get(key) - .map(|v| v.to_owned()) - .unwrap_or_else(|| std::env::var(key).unwrap_or(default)) - } - - fn u64_or_default(map: &HashMap, key: &str, default: u64) -> u64 { - str_option(map, key) - .and_then(|v| v.parse().ok()) - .unwrap_or(default) - } - - fn ensure_env_var(map: &HashMap, key: &str) { - if let Some(val) = str_option(map, key) { - std::env::set_var(key, val); - } - } -} - -impl Default for S3StorageOptions { - /// Creates an instance of S3StorageOptions from environment variables. - fn default() -> S3StorageOptions { - Self::from_map(HashMap::new()) - } -} - -impl From> for StorageError { - fn from(error: RusotoError) -> Self { - match error { - RusotoError::Service(rusoto_s3::GetObjectError::NoSuchKey(_)) => StorageError::NotFound, - _ => StorageError::S3Get { source: error }, - } - } -} - -impl From> for StorageError { - fn from(error: RusotoError) -> Self { - match error { - RusotoError::Service(rusoto_s3::HeadObjectError::NoSuchKey(_)) => { - StorageError::NotFound - } - // rusoto tries to parse response body which is missing in HEAD request - // see https://github.com/rusoto/rusoto/issues/716 - RusotoError::Unknown(r) if r.status == 404 => StorageError::NotFound, - _ => StorageError::S3Head { source: error }, - } - } -} - -impl From> for StorageError { - fn from(error: RusotoError) -> Self { - StorageError::S3Put { source: error } - } -} - -impl From> for StorageError { - fn from(error: RusotoError) -> Self { - match error { - RusotoError::Service(rusoto_s3::ListObjectsV2Error::NoSuchBucket(_)) => { - StorageError::NotFound - } - _ => StorageError::S3List { source: error }, - } - } -} - -impl From> for StorageError { - fn from(error: RusotoError) -> Self { - match error { - RusotoError::Unknown(response) if response.status == 404 => StorageError::NotFound, - _ => StorageError::S3Copy { source: error }, - } - } -} - -/// The extension of StsAssumeRoleSessionCredentialsProvider in order to provide new session_name -/// on each credentials refresh. -struct AssumeRoleCredentialsProvider { - sts_client: StsClient, - assume_role_arn: String, - session_name: Option, -} - -#[async_trait::async_trait] -impl ProvideAwsCredentials for AssumeRoleCredentialsProvider { - async fn credentials(&self) -> Result { - let session_name = self.session_name.as_deref().unwrap_or("delta-rs"); - let session_name = format!("{}-{}", session_name, Uuid::new_v4()); - let provider = StsAssumeRoleSessionCredentialsProvider::new( - self.sts_client.clone(), - self.assume_role_arn.clone(), - session_name, - None, - None, - None, - None, - ); - provider.credentials().await - } -} - -fn get_sts_assume_role_provider( - assume_role_arn: String, - options: &S3StorageOptions, -) -> Result, StorageError> { - let sts_client = StsClient::new_with( - create_http_client(options.sts_pool_idle_timeout)?, - DefaultCredentialsProvider::new()?, - options.region.clone(), - ); - - let provider = AssumeRoleCredentialsProvider { - sts_client, - assume_role_arn, - session_name: options.assume_role_session_name.clone(), - }; - - Ok(AutoRefreshingProvider::new(provider)?) -} - -fn create_http_client( - pool_idle_timeout: Duration, -) -> Result>>, StorageError> { - let mut config = HttpConfig::new(); - config.pool_idle_timeout(pool_idle_timeout); - let https_connector = HttpsConnectorBuilder::new() - .with_native_roots() - .https_or_http() - .enable_http2() - .build(); - match env::var("HTTPS_PROXY") { - Ok(proxy_uri) => { - let proxy = Proxy::new(Intercept::All, proxy_uri.parse()?); - let proxy_connector = ProxyConnector::from_proxy(https_connector, proxy)?; - Ok(HttpClient::>>::from_connector_with_config( - proxy_connector, - config, - )) - } - Err(_) => Ok( - HttpClient::>>::from_connector_with_config( - ProxyConnector::new(https_connector)?, - config, - ), - ), - } -} - -fn get_web_identity_provider() -> Result, StorageError> -{ - let provider = WebIdentityProvider::from_k8s_env(); - Ok(AutoRefreshingProvider::new(provider)?) -} - -fn create_s3_client(options: &S3StorageOptions) -> Result { - let http_client = create_http_client(options.s3_pool_idle_timeout)?; - let region = options.region.clone(); - if options.use_web_identity { - let provider = get_web_identity_provider()?; - Ok(S3Client::new_with(http_client, provider, region)) - } else if let Some(assume_role_arn) = &options.assume_role_arn { - let provider = get_sts_assume_role_provider(assume_role_arn.to_owned(), options)?; - Ok(S3Client::new_with(http_client, provider, region)) - } else { - Ok(S3Client::new_with( - http_client, - DefaultCredentialsProvider::new()?, - region, - )) - } -} - -fn parse_obj_last_modified_time( - last_modified: &Option, -) -> Result, StorageError> { - let dt_str = last_modified.as_ref().ok_or_else(|| { - StorageError::S3Generic("S3 Object missing last modified attribute".to_string()) - })?; - // last modified time in object is returned in rfc3339 format - // https://docs.aws.amazon.com/AmazonS3/latest/API/API_Object.html - let dt = DateTime::::parse_from_rfc3339(dt_str).map_err(|e| { - StorageError::S3Generic(format!( - "Failed to parse S3 modified time as rfc3339: {}, got: {:?}", - e, last_modified, - )) - })?; - - Ok(DateTime::::from(dt)) -} - -fn parse_head_obj_last_modified_time( - last_modified: &Option, -) -> Result, StorageError> { - let dt_str = last_modified.as_ref().ok_or_else(|| { - StorageError::S3Generic("S3 Object missing last modified attribute".to_string()) - })?; - // head object response sets last-modified time in rfc2822 format: - // https://docs.aws.amazon.com/AmazonS3/latest/API/API_HeadObject.html#API_HeadObject_ResponseSyntax - let dt = DateTime::::parse_from_rfc2822(dt_str).map_err(|e| { - StorageError::S3Generic(format!( - "Failed to parse S3 modified time as rfc2822: {}, got: {:?}", - e, last_modified, - )) - })?; - - Ok(DateTime::::from(dt)) -} - -fn try_object_meta_from(bucket: &str, obj: rusoto_s3::Object) -> Result { - let key = obj - .key - .ok_or_else(|| StorageError::S3Generic("S3 Object missing key attribute".to_string()))?; - - Ok(ObjectMeta { - path: format!("s3://{}/{}", bucket, key), - modified: parse_obj_last_modified_time(&obj.last_modified)?, - size: obj.size, - }) -} - -/// Struct describing an object stored in S3. -#[derive(Debug, PartialEq, Eq)] -pub struct S3Object<'a> { - /// The bucket where the object is stored. - pub bucket: &'a str, - /// The key of the object within the bucket. - pub key: &'a str, -} - -impl<'a> fmt::Display for S3Object<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "s3://{}/{}", self.bucket, self.key) - } -} - -/// An S3 implementation of the [StorageBackend] trait -/// -/// The backend can optionally use [dynamodb_lock] to better support concurrent -/// writers. To do so, either pass in a [dynamodb_lock::LockClient] to [S3StorageBackend::new_with] -/// or configure the locking client within [S3StorageOptions]. For example: -/// -/// ```rust -/// use std::collections::HashMap; -/// use deltalake::storage::s3::{S3StorageOptions, S3StorageBackend, s3_storage_options}; -/// -/// let options = S3StorageOptions::from_map([ -/// (s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"), -/// // Options passed down to dynamodb_lock::DynamoDbOptions (default values shown below) -/// ("table_name", "delta_rs_lock_table"), -/// ("partition_key_value", "delta-rs"), -/// ("owner_name", ""), // Should be unique across writers -/// ("lease_duration", "20"), // seconds -/// ("refresh_period", "1000"), // milliseconds -/// ("additional_time_to_wait_for_lock", "1000"), // milliseconds -/// ].iter().map(|(k, v)| (k.to_string(), v.to_string())).collect()); -/// let backend = S3StorageBackend::new_from_options(options); -/// ``` -pub struct S3StorageBackend { - client: rusoto_s3::S3Client, - s3_lock_client: Option, - options: S3StorageOptions, -} - -impl S3StorageBackend { - /// Creates a new S3StorageBackend. - pub fn new() -> Result { - let options = S3StorageOptions::default(); - let client = create_s3_client(&options)?; - let s3_lock_client = try_create_lock_client(&options)?; - - Ok(Self { - client, - s3_lock_client, - options, - }) - } - - /// Creates a new S3StorageBackend from the provided options. - /// - /// Options are described in [s3_storage_options]. - pub fn new_from_options(options: S3StorageOptions) -> Result { - let client = create_s3_client(&options)?; - let s3_lock_client = try_create_lock_client(&options)?; - - Ok(Self { - client, - s3_lock_client, - options, - }) - } - - /// Creates a new S3StorageBackend with given options, s3 client and lock client. - pub fn new_with( - client: rusoto_s3::S3Client, - lock_client: Option>, - options: S3StorageOptions, - ) -> Self { - let s3_lock_client = lock_client.map(|lc| S3LockClient { lock_client: lc }); - Self { - client, - s3_lock_client, - options, - } - } - - async fn unsafe_rename_obj( - self: &S3StorageBackend, - src: &str, - dst: &str, - ) -> Result<(), StorageError> { - match self.head_obj(dst).await { - Ok(_) => return Err(StorageError::AlreadyExists(dst.to_string())), - Err(StorageError::NotFound) => (), - Err(e) => return Err(e), - } - - let src = parse_uri(src)?.into_s3object()?; - let dst = parse_uri(dst)?.into_s3object()?; - - self.client - .copy_object(CopyObjectRequest { - bucket: dst.bucket.to_string(), - key: dst.key.to_string(), - copy_source: format!("{}/{}", src.bucket, src.key), - ..Default::default() - }) - .await?; - - self.client - .delete_object(DeleteObjectRequest { - bucket: src.bucket.to_string(), - key: src.key.to_string(), - ..Default::default() - }) - .await?; - - Ok(()) - } -} - -impl Default for S3StorageBackend { - fn default() -> Self { - Self::new().unwrap() - } -} - -impl std::fmt::Debug for S3StorageBackend { - fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - write!(fmt, "S3StorageBackend") - } -} - -#[async_trait::async_trait] -impl StorageBackend for S3StorageBackend { - async fn head_obj(&self, path: &str) -> Result { - let uri = parse_uri(path)?.into_s3object()?; - - let result = self - .client - .head_object(HeadObjectRequest { - bucket: uri.bucket.to_string(), - key: uri.key.to_string(), - ..Default::default() - }) - .await?; - - Ok(ObjectMeta { - path: path.to_string(), - modified: parse_head_obj_last_modified_time(&result.last_modified)?, - size: result.content_length, - }) - } - - async fn get_obj(&self, path: &str) -> Result, StorageError> { - debug!("fetching s3 object: {}...", path); - - let uri = parse_uri(path)?.into_s3object()?; - let result = get_object_with_retries( - &self.client, - uri.bucket, - uri.key, - self.options.s3_get_internal_server_error_retries, - None, - ) - .await?; - - debug!("streaming data from {}...", path); - let mut buf = Vec::new(); - let stream = result - .body - .ok_or_else(|| StorageError::S3MissingObjectBody(path.to_string()))?; - stream - .into_async_read() - .read_to_end(&mut buf) - .await - .map_err(|e| { - StorageError::S3Generic(format!("Failed to read object content: {}", e)) - })?; - - debug!("s3 object fetched: {}", path); - Ok(buf) - } - - async fn get_range(&self, path: &str, range: Range) -> Result, StorageError> { - debug!("fetching s3 object: {}...", path); - - let uri = parse_uri(path)?.into_s3object()?; - let result = get_object_with_retries( - &self.client, - uri.bucket, - uri.key, - self.options.s3_get_internal_server_error_retries, - Some(range), - ) - .await?; - - debug!("streaming data from {}...", path); - let mut buf = Vec::new(); - let stream = result - .body - .ok_or_else(|| StorageError::S3MissingObjectBody(path.to_string()))?; - stream - .into_async_read() - .read_to_end(&mut buf) - .await - .map_err(|e| { - StorageError::S3Generic(format!("Failed to read object content: {}", e)) - })?; - - debug!("s3 object fetched: {}", path); - Ok(buf) - } - - async fn list_objs<'a>( - &'a self, - path: &'a str, - ) -> Result>, StorageError> { - let uri = parse_uri(path)?.into_s3object()?; - - /// This enum is used to represent 3 states in our object metadata streaming logic: - /// * Value(None): the initial state, prior to performing any s3 list call. - /// * Value(Some(String)): s3 list call returned us a continuation token to be used in - /// subsequent list call after we got through the current page. - /// * End: previous s3 list call reached end of page, we should not perform more s3 list - /// call going forward. - enum ContinuationToken { - Value(Option), - End, - } - - struct ListContext { - client: rusoto_s3::S3Client, - obj_iter: std::vec::IntoIter, - continuation_token: ContinuationToken, - bucket: String, - key: String, - } - let ctx = ListContext { - obj_iter: Vec::new().into_iter(), - continuation_token: ContinuationToken::Value(None), - bucket: uri.bucket.to_string(), - key: uri.key.to_string(), - client: self.client.clone(), - }; - - async fn next_meta( - mut ctx: ListContext, - ) -> Option<(Result, ListContext)> { - match ctx.obj_iter.next() { - Some(obj) => Some((try_object_meta_from(&ctx.bucket, obj), ctx)), - None => match &ctx.continuation_token { - ContinuationToken::End => None, - ContinuationToken::Value(v) => { - let list_req = ListObjectsV2Request { - bucket: ctx.bucket.clone(), - prefix: Some(ctx.key.clone()), - continuation_token: v.clone(), - ..Default::default() - }; - let result = match ctx.client.list_objects_v2(list_req).await { - Ok(res) => res, - Err(e) => { - return Some((Err(e.into()), ctx)); - } - }; - ctx.continuation_token = result - .next_continuation_token - .map(|t| ContinuationToken::Value(Some(t))) - .unwrap_or(ContinuationToken::End); - ctx.obj_iter = result.contents.unwrap_or_default().into_iter(); - ctx.obj_iter - .next() - .map(|obj| (try_object_meta_from(&ctx.bucket, obj), ctx)) - } - }, - } - } - - Ok(Box::pin(futures::stream::unfold(ctx, next_meta))) - } - - async fn put_obj(&self, path: &str, obj_bytes: &[u8]) -> Result<(), StorageError> { - debug!("put s3 object: {}...", path); - - let uri = parse_uri(path)?.into_s3object()?; - let put_req = PutObjectRequest { - bucket: uri.bucket.to_string(), - key: uri.key.to_string(), - body: Some(obj_bytes.to_vec().into()), - ..Default::default() - }; - - self.client.put_object(put_req).await?; - - Ok(()) - } - - async fn rename_obj_noreplace(&self, src: &str, dst: &str) -> Result<(), StorageError> { - debug!("rename s3 object: {} -> {}...", src, dst); - - let lock_client = match self.s3_lock_client { - Some(ref lock_client) => lock_client, - None => { - return Err(StorageError::S3Generic( - "dynamodb locking is not enabled".to_string(), - )) - } - }; - - lock_client.rename_with_lock(self, src, dst).await?; - - Ok(()) - } - - async fn delete_obj(&self, path: &str) -> Result<(), StorageError> { - debug!("delete s3 object: {}...", path); - - let uri = parse_uri(path)?.into_s3object()?; - let delete_req = DeleteObjectRequest { - bucket: uri.bucket.to_string(), - key: uri.key.to_string(), - ..Default::default() - }; - - self.client.delete_object(delete_req).await?; - - Ok(()) - } - - async fn delete_objs(&self, paths: &[String]) -> Result<(), StorageError> { - debug!("delete s3 objects: {:?}...", paths); - if paths.is_empty() { - return Ok(()); - } - - let s3_objects = paths - .iter() - .map(|path| Ok(parse_uri(path)?.into_s3object()?)) - .collect::, StorageError>>()?; - - // Check whether all buckets are equal - let bucket = s3_objects[0].bucket; - s3_objects.iter().skip(1).try_for_each(|object| { - let other_bucket = object.bucket; - if other_bucket != bucket { - Err(StorageError::S3Generic( - format!("All buckets of the paths in `S3StorageBackend::delete_objs` should be the same. Expected '{}', got '{}'", bucket, other_bucket) - )) - } else { - Ok(()) - } - })?; - - // S3 has a maximum of 1000 files to delete - let chunks = s3_objects.chunks(1000); - for chunk in chunks { - let delete = Delete { - objects: chunk - .iter() - .map(|obj| ObjectIdentifier { - key: obj.key.to_string(), - ..Default::default() - }) - .collect(), - ..Default::default() - }; - let delete_req = DeleteObjectsRequest { - bucket: bucket.to_string(), - delete, - ..Default::default() - }; - self.client.delete_objects(delete_req).await?; - } - - Ok(()) - } -} - -fn try_create_lock_client( - options: &S3StorageOptions, -) -> Result, StorageError> { - let dispatcher = HttpClient::new()?; - - match &options.locking_provider { - Some(p) if p.to_lowercase() == "dynamodb" => { - let dynamodb_client = match options.use_web_identity { - true => rusoto_dynamodb::DynamoDbClient::new_with( - dispatcher, - get_web_identity_provider()?, - options.region.clone(), - ), - false => rusoto_dynamodb::DynamoDbClient::new(options.region.clone()), - }; - let lock_client = dynamodb_lock::DynamoDbLockClient::new( - dynamodb_client, - dynamodb_lock::DynamoDbOptions::from_map(options.extra_opts.clone()), - ); - Ok(Some(S3LockClient { - lock_client: Box::new(lock_client), - })) - } - _ => Ok(None), - } -} - -async fn get_object_with_retries( - client: &S3Client, - bucket: &str, - key: &str, - retries: usize, - range: Option>, -) -> Result> { - let mut tries = 0; - loop { - let result = client - .get_object(GetObjectRequest { - bucket: bucket.to_string(), - key: key.to_string(), - range: range.as_ref().map(format_http_range), - ..Default::default() - }) - .await; - match result { - Err(RusotoError::Unknown(e)) if e.status.is_server_error() && tries < retries => { - log::warn!("Got {:?}, retrying", e); - tries += 1; - continue; - } - _ => { - return result; - } - } - } -} - -fn format_http_range(range: &std::ops::Range) -> String { - format!("bytes={}-{}", range.start, range.end.saturating_sub(1)) -} - -#[cfg(test)] -mod tests { - use super::*; - - use maplit::hashmap; - use serial_test::serial; - - #[test] - fn parse_s3_object_uri() { - let uri = parse_uri("s3://foo/bar/baz").unwrap(); - assert_eq!(uri.path(), "bar/baz"); - assert_eq!( - uri.into_s3object().unwrap(), - S3Object { - bucket: "foo", - key: "bar/baz", - } - ); - } - - #[test] - #[serial] - fn storage_options_default_test() { - std::env::set_var(s3_storage_options::AWS_ENDPOINT_URL, "http://localhost"); - std::env::set_var(s3_storage_options::AWS_REGION, "us-west-1"); - std::env::set_var(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); - std::env::set_var( - s3_storage_options::AWS_S3_ASSUME_ROLE_ARN, - "arn:aws:iam::123456789012:role/some_role", - ); - std::env::set_var(s3_storage_options::AWS_S3_ROLE_SESSION_NAME, "session_name"); - std::env::set_var( - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE, - "token_file", - ); - std::env::remove_var(s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS); - std::env::remove_var(s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS); - std::env::remove_var(s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES); - - let options = S3StorageOptions::default(); - - assert_eq!( - S3StorageOptions { - _endpoint_url: Some("http://localhost".to_string()), - region: Region::Custom { - name: "us-west-1".to_string(), - endpoint: "http://localhost".to_string() - }, - aws_access_key_id: Some("test".to_string()), - aws_secret_access_key: Some("test".to_string()), - aws_session_token: None, - assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), - assume_role_session_name: Some("session_name".to_string()), - use_web_identity: true, - locking_provider: Some("dynamodb".to_string()), - s3_pool_idle_timeout: Duration::from_secs(15), - sts_pool_idle_timeout: Duration::from_secs(10), - s3_get_internal_server_error_retries: 10, - extra_opts: HashMap::new(), - }, - options - ); - } - - #[test] - #[serial] - fn storage_options_with_only_region_and_credentials() { - std::env::remove_var(s3_storage_options::AWS_ENDPOINT_URL); - let options = S3StorageOptions::from_map(hashmap! { - s3_storage_options::AWS_REGION.to_string() => "eu-west-1".to_string(), - s3_storage_options::AWS_ACCESS_KEY_ID.to_string() => "test".to_string(), - s3_storage_options::AWS_SECRET_ACCESS_KEY.to_string() => "test".to_string(), - }); - - assert_eq!( - S3StorageOptions { - _endpoint_url: None, - region: Region::default(), - aws_access_key_id: Some("test".to_string()), - aws_secret_access_key: Some("test".to_string()), - ..Default::default() - }, - options - ); - } - - #[test] - #[serial] - fn storage_options_from_map_test() { - let options = S3StorageOptions::from_map(hashmap! { - s3_storage_options::AWS_ENDPOINT_URL.to_string() => "http://localhost:1234".to_string(), - s3_storage_options::AWS_REGION.to_string() => "us-west-2".to_string(), - s3_storage_options::AWS_S3_LOCKING_PROVIDER.to_string() => "another_locking_provider".to_string(), - s3_storage_options::AWS_S3_ASSUME_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/another_role".to_string(), - s3_storage_options::AWS_S3_ROLE_SESSION_NAME.to_string() => "another_session_name".to_string(), - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "another_token_file".to_string(), - s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "1".to_string(), - s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS.to_string() => "2".to_string(), - s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES.to_string() => "3".to_string(), - }); - - assert_eq!( - S3StorageOptions { - _endpoint_url: Some("http://localhost:1234".to_string()), - region: Region::Custom { - name: "us-west-2".to_string(), - endpoint: "http://localhost:1234".to_string() - }, - aws_access_key_id: Some("test".to_string()), - aws_secret_access_key: Some("test".to_string()), - aws_session_token: None, - assume_role_arn: Some("arn:aws:iam::123456789012:role/another_role".to_string()), - assume_role_session_name: Some("another_session_name".to_string()), - use_web_identity: true, - locking_provider: Some("another_locking_provider".to_string()), - s3_pool_idle_timeout: Duration::from_secs(1), - sts_pool_idle_timeout: Duration::from_secs(2), - s3_get_internal_server_error_retries: 3, - extra_opts: HashMap::new(), - }, - options - ); - } - - #[test] - #[serial] - fn storage_options_mixed_test() { - std::env::set_var(s3_storage_options::AWS_ENDPOINT_URL, "http://localhost"); - std::env::set_var(s3_storage_options::AWS_REGION, "us-west-1"); - std::env::set_var(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); - std::env::set_var( - s3_storage_options::AWS_S3_ASSUME_ROLE_ARN, - "arn:aws:iam::123456789012:role/some_role", - ); - std::env::set_var(s3_storage_options::AWS_S3_ROLE_SESSION_NAME, "session_name"); - std::env::set_var( - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE, - "token_file", - ); - - std::env::set_var(s3_storage_options::AWS_S3_POOL_IDLE_TIMEOUT_SECONDS, "1"); - std::env::set_var(s3_storage_options::AWS_STS_POOL_IDLE_TIMEOUT_SECONDS, "2"); - std::env::set_var( - s3_storage_options::AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES, - "3", - ); - let options = S3StorageOptions::from_map(hashmap! { - s3_storage_options::AWS_REGION.to_string() => "us-west-2".to_string(), - "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), - "AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES".to_string() => "3".to_string(), - }); - - assert_eq!( - S3StorageOptions { - _endpoint_url: Some("http://localhost".to_string()), - region: Region::Custom { - name: "us-west-2".to_string(), - endpoint: "http://localhost".to_string() - }, - aws_access_key_id: Some("test".to_string()), - aws_secret_access_key: Some("test".to_string()), - aws_session_token: None, - assume_role_arn: Some("arn:aws:iam::123456789012:role/some_role".to_string()), - assume_role_session_name: Some("session_name".to_string()), - use_web_identity: true, - locking_provider: Some("dynamodb".to_string()), - s3_pool_idle_timeout: Duration::from_secs(1), - sts_pool_idle_timeout: Duration::from_secs(2), - s3_get_internal_server_error_retries: 3, - extra_opts: hashmap! { - "DYNAMO_LOCK_PARTITION_KEY_VALUE".to_string() => "my_lock".to_string(), - }, - }, - options - ); - } - #[test] - #[serial] - fn storage_options_web_identity_test() { - let _options = S3StorageOptions::from_map(hashmap! { - s3_storage_options::AWS_REGION.to_string() => "eu-west-1".to_string(), - s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE.to_string() => "web_identity_token_file".to_string(), - s3_storage_options::AWS_ROLE_ARN.to_string() => "arn:aws:iam::123456789012:role/web_identity_role".to_string(), - s3_storage_options::AWS_ROLE_SESSION_NAME.to_string() => "web_identity_session_name".to_string(), - }); - - assert_eq!( - "eu-west-1", - std::env::var(s3_storage_options::AWS_REGION).unwrap() - ); - - assert_eq!( - "web_identity_token_file", - std::env::var(s3_storage_options::AWS_WEB_IDENTITY_TOKEN_FILE).unwrap() - ); - - assert_eq!( - "arn:aws:iam::123456789012:role/web_identity_role", - std::env::var(s3_storage_options::AWS_ROLE_ARN).unwrap() - ); - - assert_eq!( - "web_identity_session_name", - std::env::var(s3_storage_options::AWS_ROLE_SESSION_NAME).unwrap() - ); - } -} diff --git a/rust/src/test_utils.rs b/rust/src/test_utils.rs new file mode 100644 index 0000000000..383bf9bdb9 --- /dev/null +++ b/rust/src/test_utils.rs @@ -0,0 +1,508 @@ +#![allow(dead_code, missing_docs)] +use crate::builder::gcp_storage_options; +use crate::DeltaTableBuilder; +use chrono::Utc; +use fs_extra::dir::{copy, CopyOptions}; +use object_store::DynObjectStore; +use serde_json::json; +use std::sync::Arc; +use tempdir::TempDir; + +pub type TestResult = Result<(), Box>; + +/// The IntegrationContext provides temporary resources to test against cloud storage services. +pub struct IntegrationContext { + integration: StorageIntegration, + bucket: String, + store: Arc, + tmp_dir: TempDir, +} + +impl IntegrationContext { + pub fn new( + integration: StorageIntegration, + ) -> Result> { + // environment variables are loaded from .env files if found. Otherwise + // default values based on the default setting of the respective emulators are set. + #[cfg(test)] + dotenv::dotenv().ok(); + + integration.prepare_env(); + + let tmp_dir = TempDir::new("")?; + // create a fresh bucket in every context. THis is done via CLI... + let bucket = match integration { + StorageIntegration::Local => tmp_dir.as_ref().to_str().unwrap().to_owned(), + _ => (format!("test-delta-table-{}", Utc::now().timestamp())), + }; + if let StorageIntegration::Google = integration { + gs_cli::prepare_env(); + let base_url = std::env::var("GOOGLE_BASE_URL")?; + let token = json!({"gcs_base_url": base_url, "disable_oauth": true, "client_email": "", "private_key": ""}); + let account_path = tmp_dir.path().join("gcs.json"); + std::fs::write(&account_path, serde_json::to_vec(&token)?)?; + set_env_if_not_set( + gcp_storage_options::SERVICE_ACCOUNT, + account_path.as_path().to_str().unwrap(), + ); + set_env_if_not_set( + gcp_storage_options::GOOGLE_SERVICE_ACCOUNT, + account_path.as_path().to_str().unwrap(), + ); + } + integration.crate_bucket(&bucket)?; + let store_uri = match integration { + StorageIntegration::Amazon => format!("s3://{}", &bucket), + StorageIntegration::Microsoft => format!("az://{}", &bucket), + StorageIntegration::Google => format!("gs://{}", &bucket), + StorageIntegration::Local => format!("file://{}", &bucket), + }; + + // the "storage_backend" will always point to the root ofg the object store. + // TODO should we provide the store via object_Store builders? + let store = match integration { + StorageIntegration::Local => Arc::new( + object_store::local::LocalFileSystem::new_with_prefix(tmp_dir.path())?, + ), + _ => DeltaTableBuilder::from_uri(store_uri) + .with_allow_http(true) + .build_storage()? + .storage_backend(), + }; + + Ok(Self { + integration, + bucket, + store, + tmp_dir, + }) + } + + pub fn new_with_tables( + integration: StorageIntegration, + tables: impl IntoIterator, + ) -> Result> { + let context = Self::new(integration)?; + for table in tables { + context.load_table(table)?; + } + Ok(context) + } + + /// Get a a reference to the root object store + pub fn object_store(&self) -> Arc { + self.store.clone() + } + + /// Get the URI for initializing a store at the root + pub fn root_uri(&self) -> String { + match self.integration { + StorageIntegration::Amazon => format!("s3://{}", &self.bucket), + StorageIntegration::Microsoft => format!("az://{}", &self.bucket), + StorageIntegration::Google => format!("gs://{}", &self.bucket), + StorageIntegration::Local => format!("file://{}", &self.bucket), + } + } + + pub fn uri_for_table(&self, table: TestTables) -> String { + format!("{}/{}", self.root_uri(), table.as_name()) + } + + pub fn load_table(&self, table: TestTables) -> TestResult { + match self.integration { + StorageIntegration::Amazon => { + s3_cli::upload_table(table.as_path().as_str(), &self.uri_for_table(table))?; + } + StorageIntegration::Microsoft => { + let uri = format!("{}/{}", self.bucket, table.as_name()); + az_cli::upload_table(&table.as_path(), &uri)?; + } + StorageIntegration::Local => { + let mut options = CopyOptions::new(); + options.content_only = true; + let dest_path = self.tmp_dir.path().join(&table.as_name()); + std::fs::create_dir_all(&dest_path)?; + copy(&table.as_path(), &dest_path, &options)?; + } + StorageIntegration::Google => todo!(), + }; + Ok(()) + } + + pub fn load_table_with_name(&self, table: TestTables, name: impl AsRef) -> TestResult { + match self.integration { + StorageIntegration::Amazon => { + s3_cli::upload_table( + table.as_path().as_str(), + &format!("{}/{}", self.root_uri(), name.as_ref()), + )?; + } + StorageIntegration::Microsoft => { + let uri = format!("{}/{}", self.bucket, name.as_ref()); + az_cli::upload_table(&table.as_path(), &uri)?; + } + StorageIntegration::Local => { + let mut options = CopyOptions::new(); + options.content_only = true; + let dest_path = self.tmp_dir.path().join(name.as_ref()); + std::fs::create_dir_all(&dest_path)?; + copy(&table.as_path(), &dest_path, &options)?; + } + StorageIntegration::Google => todo!(), + }; + Ok(()) + } +} + +impl Drop for IntegrationContext { + fn drop(&mut self) { + match self.integration { + StorageIntegration::Amazon => { + s3_cli::delete_bucket(&self.root_uri()).unwrap(); + s3_cli::delete_lock_table().unwrap(); + } + StorageIntegration::Microsoft => { + az_cli::delete_container(&self.bucket).unwrap(); + } + StorageIntegration::Google => { + gs_cli::delete_bucket(&self.bucket).unwrap(); + } + StorageIntegration::Local => (), + }; + } +} + +/// Kinds of storage integration +pub enum StorageIntegration { + Amazon, + Microsoft, + Google, + Local, +} + +impl StorageIntegration { + fn prepare_env(&self) { + match self { + Self::Microsoft => az_cli::prepare_env(), + Self::Amazon => s3_cli::prepare_env(), + Self::Google => gs_cli::prepare_env(), + Self::Local => (), + } + } + + fn crate_bucket(&self, name: impl AsRef) -> std::io::Result<()> { + match self { + Self::Microsoft => { + az_cli::create_container(name)?; + Ok(()) + } + Self::Amazon => { + s3_cli::create_bucket(&format!("s3://{}", name.as_ref()))?; + set_env_if_not_set( + "DYNAMO_LOCK_PARTITION_KEY_VALUE", + format!("s3://{}", name.as_ref()), + ); + s3_cli::create_lock_table()?; + Ok(()) + } + Self::Google => { + gs_cli::create_bucket(name)?; + Ok(()) + } + Self::Local => Ok(()), + } + } +} + +/// Reference tables from the test data folder +pub enum TestTables { + Simple, + SimpleCommit, + Golden, + Custom(String), +} + +impl TestTables { + fn as_path(&self) -> String { + // env "CARGO_MANIFEST_DIR" is "the directory containing the manifest of your package", + // set by `cargo run` or `cargo test`, see: + // https://doc.rust-lang.org/cargo/reference/environment-variables.html + let dir = env!("CARGO_MANIFEST_DIR"); + let data_path = std::path::Path::new(dir).join("tests/data"); + match self { + Self::Simple => data_path.join("simple_table").to_str().unwrap().to_owned(), + Self::SimpleCommit => data_path.join("simple_commit").to_str().unwrap().to_owned(), + Self::Golden => data_path + .join("golden/data-reader-array-primitives") + .to_str() + .unwrap() + .to_owned(), + // the data path for upload does not apply to custom tables. + Self::Custom(_) => todo!(), + } + } + + pub fn as_name(&self) -> String { + match self { + Self::Simple => "simple".into(), + Self::SimpleCommit => "simple_commit".into(), + Self::Golden => "golden".into(), + Self::Custom(name) => name.to_owned(), + } + } +} + +fn set_env_if_not_set(key: impl AsRef, value: impl AsRef) { + if std::env::var(key.as_ref()).is_err() { + std::env::set_var(key.as_ref(), value.as_ref()) + }; +} + +/// small wrapper around az cli +pub mod az_cli { + use super::set_env_if_not_set; + use crate::builder::azure_storage_options; + use std::process::{Command, ExitStatus, Stdio}; + + /// Create a new bucket + pub fn create_container(container_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "container", + "create", + "-n", + container_name.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + /// delete bucket + pub fn delete_container(container_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "container", + "delete", + "-n", + container_name.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + /// prepare_env + pub fn prepare_env() { + set_env_if_not_set(azure_storage_options::AZURE_STORAGE_USE_EMULATOR, "1"); + set_env_if_not_set( + azure_storage_options::AZURE_STORAGE_ACCOUNT_NAME, + "devstoreaccount1", + ); + set_env_if_not_set(azure_storage_options::AZURE_STORAGE_ACCOUNT_KEY, "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="); + set_env_if_not_set( + "AZURE_STORAGE_CONNECTION_STRING", + "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://localhost:10000/devstoreaccount1;" + ); + } + + pub fn upload_table(src: &str, dst: &str) -> std::io::Result { + let mut child = Command::new("az") + .args(["storage", "blob", "upload-batch", "-d", dst, "-s", src]) + .stdout(Stdio::null()) + .spawn() + .expect("az command is installed"); + child.wait() + } +} + +/// small wrapper around s3 cli +pub mod s3_cli { + use super::set_env_if_not_set; + use crate::builder::s3_storage_options; + use std::process::{Command, ExitStatus, Stdio}; + + /// Create a new bucket + pub fn create_bucket(bucket_name: impl AsRef) -> std::io::Result { + let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) + .expect("variable ENDPOINT must be set to connect to S3"); + let region = std::env::var(s3_storage_options::AWS_REGION) + .expect("variable AWS_REGION must be set to connect to S3"); + let mut child = Command::new("aws") + .args([ + "s3", + "mb", + bucket_name.as_ref(), + "--endpoint-url", + &endpoint, + "--region", + ®ion, + ]) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + /// delete bucket + pub fn delete_bucket(bucket_name: impl AsRef) -> std::io::Result { + let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) + .expect("variable ENDPOINT must be set to connect to S3"); + let mut child = Command::new("aws") + .args([ + "s3", + "rb", + bucket_name.as_ref(), + "--endpoint-url", + &endpoint, + "--force", + ]) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + /// prepare_env + pub fn prepare_env() { + set_env_if_not_set( + s3_storage_options::AWS_ENDPOINT_URL, + "http://localhost:4566", + ); + set_env_if_not_set(s3_storage_options::AWS_ACCESS_KEY_ID, "test"); + set_env_if_not_set(s3_storage_options::AWS_SECRET_ACCESS_KEY, "test"); + set_env_if_not_set("AWS_DEFAULT_REGION", "us-east-1"); + set_env_if_not_set(s3_storage_options::AWS_REGION, "us-east-1"); + set_env_if_not_set(s3_storage_options::AWS_S3_LOCKING_PROVIDER, "dynamodb"); + set_env_if_not_set("DYNAMO_LOCK_TABLE_NAME", "test_table"); + set_env_if_not_set("DYNAMO_LOCK_REFRESH_PERIOD_MILLIS", "100"); + set_env_if_not_set("DYNAMO_LOCK_ADDITIONAL_TIME_TO_WAIT_MILLIS", "100"); + } + + pub fn upload_table(src: &str, dst: &str) -> std::io::Result { + let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) + .expect("variable AWS_ENDPOINT_URL must be set to connect to S3 emulator"); + let mut child = Command::new("aws") + .args([ + "s3", + "sync", + src, + dst, + "--delete", + "--endpoint-url", + &endpoint, + ]) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + pub fn create_lock_table() -> std::io::Result { + let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) + .expect("variable AWS_ENDPOINT_URL must be set to connect to S3 emulator"); + let table_name = + std::env::var("DYNAMO_LOCK_TABLE_NAME").unwrap_or_else(|_| "test_table".into()); + let mut child = Command::new("aws") + .args([ + "dynamodb", + "create-table", + "--table-name", + &table_name, + "--endpoint-url", + &endpoint, + "--attribute-definitions", + "AttributeName=key,AttributeType=S", + "--key-schema", + "AttributeName=key,KeyType=HASH", + "--provisioned-throughput", + "ReadCapacityUnits=10,WriteCapacityUnits=10", + ]) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + child.wait() + } + + pub fn delete_lock_table() -> std::io::Result { + let endpoint = std::env::var(s3_storage_options::AWS_ENDPOINT_URL) + .expect("variable AWS_ENDPOINT_URL must be set to connect to S3 emulator"); + let table_name = + std::env::var("DYNAMO_LOCK_TABLE_NAME").unwrap_or_else(|_| "test_table".into()); + let mut child = Command::new("aws") + .args([ + "dynamodb", + "delete-table", + "--table-name", + &table_name, + "--endpoint-url", + &endpoint, + ]) + .stdout(Stdio::null()) + .spawn() + .expect("aws command is installed"); + child.wait() + } +} + +/// small wrapper around google api +pub mod gs_cli { + use crate::gcp_storage_options; + + use super::set_env_if_not_set; + use serde_json::json; + use std::process::{Command, ExitStatus}; + + pub fn create_bucket(container_name: impl AsRef) -> std::io::Result { + let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") + .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); + let payload = json!({ "name": container_name.as_ref() }); + let mut child = Command::new("curl") + .args([ + "--insecure", + "-v", + "-X", + "POST", + "--data-binary", + &format!("'{}'", &serde_json::to_string(&payload)?), + "-H", + "Content-Type: application/json", + &endpoint, + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + pub fn delete_bucket(container_name: impl AsRef) -> std::io::Result { + let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") + .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); + let payload = json!({ "name": container_name.as_ref() }); + let mut child = Command::new("curl") + .args([ + "--insecure", + "-v", + "-X", + "DELETE", + "--data-binary", + &serde_json::to_string(&payload)?, + "-H", + "Content-Type: application/json", + &endpoint, + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + pub fn upload_table(_src: &str, _dst: &str) -> std::io::Result { + todo!() + } + + /// prepare_env + pub fn prepare_env() { + set_env_if_not_set(gcp_storage_options::GOOGLE_USE_EMULATOR, "1"); + set_env_if_not_set("GOOGLE_BASE_URL", "https://localhost:4443"); + set_env_if_not_set("GOOGLE_ENDPOINT_URL", "https://localhost:4443/storage/v1/b"); + } +} diff --git a/rust/src/writer/json.rs b/rust/src/writer/json.rs index 42b5f44ca7..698640ae58 100644 --- a/rust/src/writer/json.rs +++ b/rust/src/writer/json.rs @@ -7,8 +7,9 @@ use super::{ }, DeltaWriter, DeltaWriterError, }; +use crate::builder::DeltaTableBuilder; use crate::{action::Add, DeltaTable, DeltaTableMetaData, Schema}; -use crate::{object_store::DeltaObjectStore, writer::utils::ShareableBuffer}; +use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; use arrow::{ datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}, record_batch::*, @@ -184,7 +185,9 @@ impl JsonWriter { partition_columns: Option>, storage_options: Option>, ) -> Result { - let storage = DeltaObjectStore::try_new_with_options(&table_uri, storage_options)?; + let storage = DeltaTableBuilder::from_uri(&table_uri) + .with_storage_options(storage_options.unwrap_or_default()) + .build_storage()?; // Initialize writer properties for the underlying arrow writer let writer_properties = WriterProperties::builder() @@ -193,7 +196,7 @@ impl JsonWriter { .build(); Ok(Self { - storage: Arc::new(storage), + storage, arrow_schema_ref: schema, writer_properties, partition_columns: partition_columns.unwrap_or_default(), diff --git a/rust/src/writer/mod.rs b/rust/src/writer/mod.rs index 3c036db692..b94ead4537 100644 --- a/rust/src/writer/mod.rs +++ b/rust/src/writer/mod.rs @@ -2,6 +2,8 @@ // TODO // - consider file size when writing parquet files // - handle writer version +#![cfg(all(feature = "arrow", feature = "parquet"))] + pub mod json; pub mod record_batch; mod stats; @@ -12,7 +14,7 @@ pub mod utils; use crate::{ action::{Action, Add, ColumnCountStat, Stats}, delta::DeltaTable, - DeltaDataTypeVersion, DeltaTableError, StorageError, UriError, + DeltaDataTypeVersion, DeltaTableError, }; use arrow::{datatypes::SchemaRef, datatypes::*, error::ArrowError}; use async_trait::async_trait; @@ -64,22 +66,6 @@ pub enum DeltaWriterError { stats: Stats, }, - /// Invalid table paths was specified for the delta table. - #[error("Invalid table path: {}", .source)] - UriError { - /// The wrapped [`UriError`]. - #[from] - source: UriError, - }, - - /// deltalake storage backend returned an error. - #[error("Storage interaction failed: {source}")] - Storage { - /// The wrapped [`StorageError`] - #[from] - source: StorageError, - }, - /// underlying object store returned an error. #[error("ObjectStore interaction failed: {source}")] ObjectStore { diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index 495d070861..77f400b71f 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -34,9 +34,10 @@ use super::{ }, DeltaWriter, DeltaWriterError, }; +use crate::builder::DeltaTableBuilder; use crate::writer::stats::apply_null_counts; use crate::writer::utils::ShareableBuffer; -use crate::{action::Add, object_store::DeltaObjectStore, DeltaTable, DeltaTableMetaData, Schema}; +use crate::{action::Add, storage::DeltaObjectStore, DeltaTable, DeltaTableMetaData, Schema}; use arrow::record_batch::RecordBatch; use arrow::{ array::{Array, UInt32Array}, @@ -75,7 +76,9 @@ impl RecordBatchWriter { partition_columns: Option>, storage_options: Option>, ) -> Result { - let storage = DeltaObjectStore::try_new_with_options(&table_uri, storage_options)?; + let storage = DeltaTableBuilder::from_uri(&table_uri) + .with_storage_options(storage_options.unwrap_or_default()) + .build_storage()?; // Initialize writer properties for the underlying arrow writer let writer_properties = WriterProperties::builder() @@ -84,7 +87,7 @@ impl RecordBatchWriter { .build(); Ok(Self { - storage: Arc::new(storage), + storage, arrow_schema_ref: schema, writer_properties, partition_columns: partition_columns.unwrap_or_default(), @@ -493,7 +496,8 @@ mod tests { String::from("modified=2021-02-02/id=A"), String::from("modified=2021-02-02/id=B"), ]; - let table_dir = Path::new(&table.table_uri); + let table_uri = table.table_uri(); + let table_dir = Path::new(&table_uri); for key in expected_keys { let partition_dir = table_dir.join(key); assert!(partition_dir.exists()) diff --git a/rust/src/writer/stats.rs b/rust/src/writer/stats.rs index 29f6640ad3..404f388c67 100644 --- a/rust/src/writer/stats.rs +++ b/rust/src/writer/stats.rs @@ -404,6 +404,7 @@ mod tests { use super::{test_utils::get_record_batch, utils::record_batch_from_message}; use crate::{ action::{ColumnCountStat, ColumnValueStat}, + builder::DeltaTableBuilder, DeltaTable, DeltaTableError, }; use lazy_static::lazy_static; @@ -540,10 +541,10 @@ mod tests { table_uri: &str, options: HashMap, ) -> Result { - let backend = crate::get_backend_for_uri_with_options(table_uri, options)?; - let mut table = DeltaTable::new(table_uri, backend, crate::DeltaTableConfig::default())?; - table.load().await?; - Ok(table) + DeltaTableBuilder::from_uri(table_uri) + .with_storage_options(options) + .load() + .await } fn create_temp_table(table_path: &Path) { diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index f76e4d23c8..d74533bb06 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -1,7 +1,7 @@ //! Utilities for writing unit tests use super::*; use crate::{ - action::Protocol, schema::Schema, DeltaTable, DeltaTableConfig, DeltaTableMetaData, + action::Protocol, schema::Schema, DeltaTable, DeltaTableBuilder, DeltaTableMetaData, SchemaDataType, SchemaField, }; use arrow::record_batch::RecordBatch; @@ -165,15 +165,9 @@ pub fn get_delta_metadata(partition_cols: &[String]) -> DeltaTableMetaData { pub fn create_bare_table() -> DeltaTable { let table_dir = tempfile::tempdir().unwrap(); let table_path = table_dir.path(); - let backend = Arc::new(crate::storage::file::FileStorageBackend::new( - table_path.to_str().unwrap(), - )); - DeltaTable::new( - table_path.to_str().unwrap(), - backend, - DeltaTableConfig::default(), - ) - .unwrap() + DeltaTableBuilder::from_uri(table_path.to_str().unwrap()) + .build() + .unwrap() } pub async fn create_initialized_table(partition_cols: &[String]) -> DeltaTable { diff --git a/rust/tests/adls_gen2_backend_test.rs b/rust/tests/adls_gen2_backend_test.rs deleted file mode 100644 index 0c67be77f1..0000000000 --- a/rust/tests/adls_gen2_backend_test.rs +++ /dev/null @@ -1,282 +0,0 @@ -#[cfg(feature = "azure")] -/// An Azure Data Lake Gen2 Storage Account is required to run these tests and must be provided by -/// the developer. Because of this requirement, the tests cannot run in CI and are therefore marked -/// #[ignore]. As a result, the developer must execute these tests on their machine. -/// In order to execute tests, remove the desired #[ignore] below and execute via: -/// 'cargo test --features azure --test adls_gen2_backend_test -- --nocapture' -/// `AZURE_STORAGE_ACCOUNT_NAME` is required to be set in the environment. -/// `AZURE_STORAGE_ACCOUNT_KEY` is required to be set in the environment. -mod adls_gen2_backend { - use azure_storage::storage_shared_key_credential::StorageSharedKeyCredential; - use azure_storage_datalake::clients::{DataLakeClient, FileSystemClient}; - use chrono::Utc; - use deltalake::{StorageBackend, StorageError}; - use futures::TryStreamExt; - use serial_test::serial; - use std::env; - use std::sync::Arc; - - #[ignore] - #[tokio::test] - #[serial] - async fn test_put() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-put"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - // Act - let file_path = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - let file_contents = &[12, 13, 14]; - let result = backend.put_obj(file_path, file_contents).await; - - // Assert - result.unwrap(); - let downloaded_file_contents = backend.get_obj(file_path).await.unwrap(); - assert_eq!(downloaded_file_contents, file_contents.to_vec()); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_put_overwrite() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-put-overwrite"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - backend.put_obj(file_path, &[12, 13, 14]).await.unwrap(); - - // Act - let file_contents = &[15, 16, 17]; - let result = backend.put_obj(file_path, file_contents).await; - - // Assert - result.unwrap(); - let downloaded_file_contents = backend.get_obj(file_path).await.unwrap(); - assert_eq!(downloaded_file_contents, file_contents.to_vec()); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_head_of_missing_file() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-head-of-missing-file"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - // Act - let file_path = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - let result = backend.head_obj(file_path).await; - - // Assert - let head_err = result.err().unwrap(); - assert!(matches!(head_err, StorageError::NotFound)); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_head_of_existing_file() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-head-of-existing-file"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - backend.put_obj(file_path, &[12, 13, 14]).await.unwrap(); - - // Act - let result = backend.head_obj(file_path).await; - - // Assert - let file_meta_data = result.unwrap(); - assert_eq!(file_meta_data.path, *file_path); - assert_eq!(file_meta_data.size, Some(3)); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_delete_existing_file() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-delete-existing-file"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - backend.put_obj(file_path, &[12, 13, 14]).await.unwrap(); - - // Act - let result = backend.delete_obj(file_path).await; - - // Assert - result.unwrap(); - let head_err = backend.head_obj(file_path).await.err().unwrap(); - assert!(matches!(head_err, StorageError::NotFound)); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_get() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-get"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - let file_contents = &[12, 13, 14]; - backend.put_obj(file_path, file_contents).await.unwrap(); - - // Act - let downloaded_file_contents = backend.get_obj(file_path).await.unwrap(); - assert_eq!(downloaded_file_contents, file_contents.to_vec()); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_rename_noreplace_succeeds() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-rename-noreplace-succeeds"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path1 = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - let file_contents = &[12, 13, 14]; - backend.put_obj(file_path1, file_contents).await.unwrap(); - - let file_path2 = &format!("{}dir1/file2-{}.txt", table_uri, Utc::now().timestamp()); - - // Act - let result = backend.rename_obj_noreplace(file_path1, file_path2).await; - - // Assert - result.unwrap(); - let downloaded_file_contents = backend.get_obj(file_path2).await.unwrap(); - assert_eq!(downloaded_file_contents, file_contents.to_vec()); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_rename_noreplace_fails() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-rename-noreplace-fails"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path1 = &format!("{}dir1/file1-{}.txt", table_uri, Utc::now().timestamp()); - backend.put_obj(file_path1, &[12, 13, 14]).await.unwrap(); - - let file_path2 = &format!("{}dir1/file2-{}.txt", table_uri, Utc::now().timestamp()); - backend.put_obj(file_path2, &[12, 13, 14]).await.unwrap(); - - // Act - let result = backend.rename_obj_noreplace(file_path1, file_path2).await; - - // Assert - let rename_obj_noreplace_error = result.err().unwrap(); - assert!( - matches!(rename_obj_noreplace_error, StorageError::AlreadyExists(path) if path == *file_path2) - ); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn test_list_objs() { - // Arrange - let file_system_prefix = "test-adls-gen2-backend-list-objs"; - let file_system_name = format!("{}-{}", file_system_prefix, Utc::now().timestamp()); - let (file_system_client, table_uri, backend) = setup(&file_system_name).await; - - let file_path = &format!("{}dir1/file1-1.txt", table_uri); - let file_contents = &[12, 13, 14]; - backend.put_obj(file_path, file_contents).await.unwrap(); - - let file_path = &format!("{}dir1/file1-2.txt", table_uri); - let file_contents = &[12, 13, 14]; - backend.put_obj(file_path, file_contents).await.unwrap(); - - // Act - let dir_path = &format!("{}dir1/", table_uri); - let files = backend - .list_objs(dir_path) - .await - .unwrap() - .try_collect::>() - .await - .unwrap(); - assert_eq!(files.len(), 2); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - async fn setup( - file_system_name: &String, - ) -> (FileSystemClient, String, Arc) { - let storage_account_name = env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let storage_account_key = env::var("AZURE_STORAGE_ACCOUNT_KEY").unwrap(); - - let file_system_client = create_file_system_client( - &storage_account_name, - &storage_account_key, - &file_system_name, - ) - .await; - - let table_uri = &format!("adls2://{}/{}/", storage_account_name, file_system_name); - let backend = deltalake::get_backend_for_uri(table_uri).unwrap(); - - (file_system_client, table_uri.to_owned(), backend) - } - - async fn create_file_system_client( - storage_account_name: &String, - storage_account_key: &String, - file_system_name: &String, - ) -> FileSystemClient { - let data_lake_client = DataLakeClient::new( - StorageSharedKeyCredential::new( - storage_account_name.to_owned(), - storage_account_key.to_owned(), - ), - None, - ); - - let file_system_client = - data_lake_client.into_file_system_client(file_system_name.to_owned()); - file_system_client.create().into_future().await.unwrap(); - - file_system_client - } -} diff --git a/rust/tests/adls_gen2_table_test.rs b/rust/tests/adls_gen2_table_test.rs deleted file mode 100644 index dd41835bb0..0000000000 --- a/rust/tests/adls_gen2_table_test.rs +++ /dev/null @@ -1,223 +0,0 @@ -#[cfg(feature = "azure")] -/// An Azure Data Lake Gen2 Storage Account is required to run these tests and must be provided by -/// the developer. Because of this requirement, the tests cannot run in CI and are therefore marked -/// #[ignore]. As a result, the developer must execute these tests on their machine. -/// In order to execute tests, remove the desired #[ignore] below and execute via: -/// 'cargo test --features azure --test adls_gen2_table_test -- --nocapture' -/// `AZURE_STORAGE_ACCOUNT_NAME` is required to be set in the environment. -/// `AZURE_STORAGE_ACCOUNT_KEY` is required to be set in the environment. -mod adls_gen2_table { - use azure_storage::storage_shared_key_credential::StorageSharedKeyCredential; - use azure_storage_datalake::prelude::DataLakeClient; - use chrono::Utc; - use deltalake::storage::azure::azure_storage_options; - use deltalake::{ - action, DeltaTable, DeltaTableConfig, DeltaTableMetaData, Schema, SchemaDataType, - SchemaField, - }; - use object_store::path::Path; - use serial_test::serial; - use std::collections::HashMap; - use std::env; - - /* - * This test requires that a file system with the name "simple" exists within the - * Storage Account and that the contents of rust/tests/data/simple_table are uploaded into - * that file system. - */ - #[ignore] - #[tokio::test] - #[serial] - async fn read_simple_table() { - let account = std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let table = deltalake::open_table(format!("adls2://{}/simple/", account).as_str()) - .await - .unwrap(); - - assert_eq!(table.version(), 4); - assert_eq!(table.get_min_writer_version(), 2); - assert_eq!(table.get_min_reader_version(), 1); - assert_eq!( - table.get_files(), - vec![ - Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), - Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), - Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), - Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), - Path::from("part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"), - ] - ); - - let tombstones = table.get_state().all_tombstones(); - assert_eq!(tombstones.len(), 31); - let remove = deltalake::action::Remove { - path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), - deletion_timestamp: Some(1587968596250), - data_change: true, - ..Default::default() - }; - assert!(tombstones.contains(&remove)); - } - - #[ignore] - #[tokio::test] - #[serial] - async fn read_simple_table_with_service_principal() { - let account = std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let client_id = std::env::var("AZURE_CLIENT_ID").unwrap(); - let client_secret = std::env::var("AZURE_CLIENT_SECRET").unwrap(); - let tenant_id = std::env::var("AZURE_TENANT_ID").unwrap(); - let mut options = std::collections::HashMap::new(); - options.insert( - azure_storage_options::AZURE_CLIENT_ID.to_string(), - client_id, - ); - options.insert( - azure_storage_options::AZURE_CLIENT_SECRET.to_string(), - client_secret, - ); - options.insert( - azure_storage_options::AZURE_TENANT_ID.to_string(), - tenant_id, - ); - - let table_uri = format!("adls2://{}/simple/", account); - let mut builder = deltalake::DeltaTableBuilder::from_uri(&table_uri).unwrap(); - let backend = deltalake::get_backend_for_uri_with_options(&table_uri, options).unwrap(); - builder = builder.with_storage_backend(backend); - - let table = builder.load().await.unwrap(); - - assert_eq!(table.version(), 4); - assert_eq!(table.get_min_writer_version(), 2); - assert_eq!(table.get_min_reader_version(), 1); - assert_eq!( - table.get_files(), - vec![ - Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), - Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), - Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), - Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), - Path::from("part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet"), - ] - ); - - let tombstones = table.get_state().all_tombstones(); - assert_eq!(tombstones.len(), 31); - let remove = deltalake::action::Remove { - path: "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet".to_string(), - deletion_timestamp: Some(1587968596250), - data_change: true, - ..Default::default() - }; - assert!(tombstones.contains(&remove)); - } - - /* - * This test has no prerequisites. - */ - #[ignore] - #[tokio::test] - #[serial] - async fn create_table_and_commit() { - // Arrange - let storage_account_name = env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let storage_account_key = env::var("AZURE_STORAGE_ACCOUNT_KEY").unwrap(); - - let data_lake_client = DataLakeClient::new( - StorageSharedKeyCredential::new( - storage_account_name.to_owned(), - storage_account_key.to_owned(), - ), - None, - ); - - // Create a new file system for test isolation - let file_system_name = format!("test-delta-table-{}", Utc::now().timestamp()); - let file_system_client = - data_lake_client.into_file_system_client(file_system_name.to_owned()); - file_system_client.create().into_future().await.unwrap(); - - let table_uri = &format!("adls2://{}/{}/", storage_account_name, file_system_name); - let backend = deltalake::get_backend_for_uri(table_uri).unwrap(); - let mut dt = DeltaTable::new(table_uri, backend, DeltaTableConfig::default()).unwrap(); - let (metadata, protocol) = table_info(); - - // Act 1 - dt.create(metadata.clone(), protocol.clone(), None, None) - .await - .unwrap(); - - // Assert 1 - assert_eq!(0, dt.version()); - assert_eq!(1, dt.get_min_reader_version()); - assert_eq!(2, dt.get_min_writer_version()); - assert_eq!(0, dt.get_files().len()); - assert_eq!(table_uri.trim_end_matches('/').to_string(), dt.table_uri); - - // Act 2 - let mut tx = dt.create_transaction(None); - tx.add_actions(tx_actions()); - let version = tx.commit(None, None).await.unwrap(); - - // Assert 2 - assert_eq!(1, version); - assert_eq!(version, dt.version()); - assert_eq!(2, dt.get_files().len()); - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); - } - - fn table_info() -> (DeltaTableMetaData, action::Protocol) { - let schema = Schema::new(vec![SchemaField::new( - "Id".to_string(), - SchemaDataType::primitive("integer".to_string()), - true, - HashMap::new(), - )]); - - let metadata = DeltaTableMetaData::new( - Some("Azure Test Table".to_string()), - None, - None, - schema, - vec![], - HashMap::new(), - ); - - let protocol = action::Protocol { - min_reader_version: 1, - min_writer_version: 2, - }; - - (metadata, protocol) - } - - fn tx_actions() -> Vec { - vec![ - action::Action::add(action::Add { - path: String::from("non-existent-file1.snappy.parquet"), - size: 396, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524294000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - }), - action::Action::add(action::Add { - path: String::from("non-existent-file2.snappy.parquet"), - size: 400, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524294000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - }), - ] - } -} diff --git a/rust/tests/checkpoint_writer_test.rs b/rust/tests/checkpoint_writer.rs similarity index 99% rename from rust/tests/checkpoint_writer_test.rs rename to rust/tests/checkpoint_writer.rs index 42b714fbfb..0274fe5dc9 100644 --- a/rust/tests/checkpoint_writer_test.rs +++ b/rust/tests/checkpoint_writer.rs @@ -108,7 +108,7 @@ mod delete_expired_delta_log_in_checkpoint { ) .await; - let table_path = table.table_uri.clone(); + let table_path = table.table_uri(); let set_file_last_modified = |version: usize, last_modified_millis: i64| { let last_modified_secs = last_modified_millis / 1000; let path = format!("{}/_delta_log/{:020}.json", &table_path, version); @@ -132,7 +132,7 @@ mod delete_expired_delta_log_in_checkpoint { table.load_version(2).await.expect("Cannot load version 2"); checkpoints::create_checkpoint_from_table_uri_and_cleanup( - &table.table_uri, + &table.table_uri(), table.version(), None, ) @@ -181,7 +181,7 @@ mod delete_expired_delta_log_in_checkpoint { table.load_version(1).await.expect("Cannot load version 1"); checkpoints::create_checkpoint_from_table_uri_and_cleanup( - &table.table_uri, + &table.table_uri(), table.version(), None, ) diff --git a/rust/tests/optimize_test.rs b/rust/tests/command_optimize.rs similarity index 98% rename from rust/tests/optimize_test.rs rename to rust/tests/command_optimize.rs index 1d82ed7772..ec1f89f218 100644 --- a/rust/tests/optimize_test.rs +++ b/rust/tests/command_optimize.rs @@ -13,10 +13,10 @@ mod optimize { use deltalake::{ action, action::Remove, - get_backend_for_uri_with_options, + builder::DeltaTableBuilder, optimize::{create_merge_plan, Optimize}, writer::{DeltaWriter, RecordBatchWriter}, - DeltaTableConfig, DeltaTableMetaData, PartitionFilter, + DeltaTableMetaData, PartitionFilter, }; use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; use rand::prelude::*; @@ -70,9 +70,8 @@ mod optimize { let tmp_dir = tempdir::TempDir::new("opt_table").unwrap(); let p = tmp_dir.path().to_str().to_owned().unwrap(); + let mut dt = DeltaTableBuilder::from_uri(p).build()?; - let backend = get_backend_for_uri_with_options(&p, HashMap::new())?; - let mut dt = DeltaTable::new(&p, backend, DeltaTableConfig::default())?; let mut commit_info = Map::::new(); let protocol = action::Protocol { diff --git a/rust/tests/vacuum_test.rs b/rust/tests/command_vacuum.rs similarity index 75% rename from rust/tests/vacuum_test.rs rename to rust/tests/command_vacuum.rs index eaeb77f087..665d341c32 100644 --- a/rust/tests/vacuum_test.rs +++ b/rust/tests/command_vacuum.rs @@ -1,7 +1,7 @@ use chrono::Duration; -use deltalake::storage::StorageError; use deltalake::vacuum::Clock; use deltalake::vacuum::Vacuum; +use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use std::sync::Arc; use common::clock::TestClock; @@ -81,13 +81,16 @@ async fn test_non_partitioned_table() { .await; let clock = TestClock::from_systemtime(); - let paths = ["delete_me.parquet", "dont_delete_me.parquet"]; + let paths = [ + Path::from("delete_me.parquet"), + Path::from("dont_delete_me.parquet"), + ]; for path in paths { context .add_file( - path, - "random junk".as_ref(), + &path, + "random junk".as_bytes().into(), &[], clock.current_timestamp_millis(), true, @@ -110,8 +113,8 @@ async fn test_non_partitioned_table() { }; assert_eq!(res.files_deleted.len(), 1); - assert!(is_deleted(&mut context, "delete_me.parquet").await); - assert!(!is_deleted(&mut context, "dont_delete_me.parquet").await); + assert!(is_deleted(&mut context, &Path::from("delete_me.parquet")).await); + assert!(!is_deleted(&mut context, &Path::from("dont_delete_me.parquet")).await); } #[tokio::test] @@ -124,16 +127,16 @@ async fn test_partitioned_table() { let clock = TestClock::from_systemtime(); let paths = [ - "date=2022-07-03/x=2/delete_me.parquet", - "date=2022-07-03/x=2/dont_delete_me.parquet", + Path::from("date=2022-07-03/x=2/delete_me.parquet"), + Path::from("date=2022-07-03/x=2/dont_delete_me.parquet"), ]; let partition_values = [("date", Some("2022-07-03")), ("x", Some("2"))]; for path in paths { context .add_file( - path, - "random junk".as_ref(), + &path, + "random junk".as_bytes().into(), &partition_values, clock.current_timestamp_millis(), true, @@ -160,8 +163,20 @@ async fn test_partitioned_table() { }; assert_eq!(res.files_deleted.len(), 1); - assert!(is_deleted(&mut context, "date=2022-07-03/x=2/delete_me.parquet").await); - assert!(!is_deleted(&mut context, "date=2022-07-03/x=2/dont_delete_me.parquet").await); + assert!( + is_deleted( + &mut context, + &Path::from("date=2022-07-03/x=2/delete_me.parquet") + ) + .await + ); + assert!( + !is_deleted( + &mut context, + &Path::from("date=2022-07-03/x=2/dont_delete_me.parquet") + ) + .await + ); } #[tokio::test] @@ -174,8 +189,8 @@ async fn test_partitions_included() { let clock = TestClock::from_systemtime(); let paths = [ - "_date=2022-07-03/delete_me.parquet", - "_date=2022-07-03/dont_delete_me.parquet", + Path::from("_date=2022-07-03/delete_me.parquet"), + Path::from("_date=2022-07-03/dont_delete_me.parquet"), ]; let partition_values = &[("_date", Some("2022-07-03"))]; @@ -183,8 +198,8 @@ async fn test_partitions_included() { for path in paths { context .add_file( - path, - "random junk".as_ref(), + &path, + "random junk".as_bytes().into(), partition_values, clock.current_timestamp_millis(), true, @@ -211,8 +226,20 @@ async fn test_partitions_included() { }; assert_eq!(res.files_deleted.len(), 1); - assert!(is_deleted(&mut context, "_date=2022-07-03/delete_me.parquet").await); - assert!(!is_deleted(&mut context, "_date=2022-07-03/dont_delete_me.parquet").await); + assert!( + is_deleted( + &mut context, + &Path::from("_date=2022-07-03/delete_me.parquet") + ) + .await + ); + assert!( + !is_deleted( + &mut context, + &Path::from("_date=2022-07-03/dont_delete_me.parquet") + ) + .await + ); } #[ignore] @@ -228,28 +255,28 @@ async fn test_non_managed_files() { let clock = TestClock::from_systemtime(); let paths_delete = vec![ - "garbage_file", - "nested/garbage_file", - "nested2/really/deep/garbage_file", + Path::from("garbage_file"), + Path::from("nested/garbage_file"), + Path::from("nested2/really/deep/garbage_file"), ]; let paths_ignore = vec![ - ".dotfile", - "_underscore", - "nested/.dotfile", - "nested2/really/deep/_underscore", + Path::from(".dotfile"), + Path::from("_underscore"), + Path::from("nested/.dotfile"), + Path::from("nested2/really/deep/_underscore"), // Directories - "_underscoredir/dont_delete_me", - "_dotdir/dont_delete_me", - "nested3/_underscoredir/dont_delete_me", - "nested4/really/deep/.dotdir/dont_delete_me", + Path::from("_underscoredir/dont_delete_me"), + Path::from("_dotdir/dont_delete_me"), + Path::from("nested3/_underscoredir/dont_delete_me"), + Path::from("nested4/really/deep/.dotdir/dont_delete_me"), ]; for path in paths_delete.iter().chain(paths_ignore.iter()) { context .add_file( path, - "random junk".as_ref(), + "random junk".as_bytes().into(), &[], clock.current_timestamp_millis(), false, @@ -283,21 +310,19 @@ async fn test_non_managed_files() { assert_eq!(res.files_deleted.len(), paths_delete.len()); for path in paths_delete { - assert!(is_deleted(&mut context, path).await); + assert!(is_deleted(&mut context, &path).await); } for path in paths_ignore { - assert!(!is_deleted(&mut context, path).await); + assert!(!is_deleted(&mut context, &path).await); } } -async fn is_deleted(context: &mut TestContext, path: &str) -> bool { - let uri = context.table.as_ref().unwrap().table_uri.clone(); +async fn is_deleted(context: &mut TestContext, path: &Path) -> bool { let backend = context.get_storage(); - let path = uri + "/" + path; - let res = backend.head_obj(&path).await; + let res = backend.head(path).await; match res { - Err(StorageError::NotFound) => true, + Err(ObjectStoreError::NotFound { .. }) => true, _ => false, } } diff --git a/rust/tests/common/adls.rs b/rust/tests/common/adls.rs index c3946545df..f52e9d4ed2 100644 --- a/rust/tests/common/adls.rs +++ b/rust/tests/common/adls.rs @@ -1,11 +1,8 @@ +use super::TestContext; use chrono::Utc; -use std::collections::HashMap; - -use azure_storage::storage_shared_key_credential::StorageSharedKeyCredential; -use azure_storage_datalake::clients::DataLakeClient; use rand::Rng; - -use super::TestContext; +use std::collections::HashMap; +use std::process::Command; pub struct AzureGen2 { account_name: String, @@ -15,27 +12,8 @@ pub struct AzureGen2 { impl Drop for AzureGen2 { fn drop(&mut self) { - let storage_account_name = self.account_name.clone(); - let storage_account_key = self.account_key.clone(); let file_system_name = self.file_system_name.clone(); - - let thread_handle = std::thread::spawn(move || { - let runtime = tokio::runtime::Runtime::new().unwrap(); - let data_lake_client = DataLakeClient::new( - StorageSharedKeyCredential::new( - storage_account_name.to_owned(), - storage_account_key.to_owned(), - ), - None, - ); - let file_system_client = - data_lake_client.into_file_system_client(file_system_name.to_owned()); - runtime - .block_on(file_system_client.delete().into_future()) - .unwrap(); - }); - - thread_handle.join().unwrap(); + az_cli::delete_container(file_system_name).unwrap(); } } @@ -44,21 +22,15 @@ pub async fn setup_azure_gen2_context() -> TestContext { let storage_account_name = std::env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); let storage_account_key = std::env::var("AZURE_STORAGE_ACCOUNT_KEY").unwrap(); + let storage_container_name = + std::env::var("AZURE_STORAGE_CONTAINER_NAME").unwrap_or("deltars".to_string()); - let data_lake_client = DataLakeClient::new( - StorageSharedKeyCredential::new( - storage_account_name.to_owned(), - storage_account_key.to_owned(), - ), - None, - ); let rand: u16 = rand::thread_rng().gen(); let file_system_name = format!("delta-rs-test-{}-{}", Utc::now().timestamp(), rand); - let file_system_client = data_lake_client.into_file_system_client(file_system_name.to_owned()); - file_system_client.create().into_future().await.unwrap(); + az_cli::create_container(&file_system_name).unwrap(); - let table_uri = format!("adls2://{}/{}/", storage_account_name, file_system_name); + let table_uri = format!("azure://{}/", file_system_name); config.insert("URI".to_string(), table_uri); config.insert( @@ -80,3 +52,46 @@ pub async fn setup_azure_gen2_context() -> TestContext { ..TestContext::default() } } + +pub mod az_cli { + use deltalake::builder::azure_storage_options; + use std::process::{Command, ExitStatus}; + + /// Create a new bucket + pub fn create_container(container_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "container", + "create", + "-n", + container_name.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + /// delete bucket + pub fn delete_container(container_name: impl AsRef) -> std::io::Result { + let mut child = Command::new("az") + .args([ + "storage", + "container", + "delete", + "-n", + container_name.as_ref(), + ]) + .spawn() + .expect("az command is installed"); + child.wait() + } + + pub fn upload_table(src: &str, dst: &str) -> std::io::Result { + let mut child = Command::new("az") + .args(["storage", "blob", "upload-batch", "-d", dst, "-s", src]) + .spawn() + .expect("az command is installed"); + child.wait() + } +} diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 540c2f1313..bb95aeeada 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -1,14 +1,15 @@ -use std::any::Any; -use tempdir::TempDir; - -use deltalake::action; -use deltalake::action::{Add, Remove}; -use deltalake::get_backend_for_uri_with_options; -use deltalake::StorageBackend; +use bytes::Bytes; +use deltalake::action::{self, Add, Remove}; +use deltalake::builder::DeltaTableBuilder; +use deltalake::storage::DeltaObjectStore; use deltalake::{DeltaTable, DeltaTableConfig, DeltaTableMetaData, Schema}; +use object_store::{path::Path, ObjectStore}; use serde_json::{Map, Value}; +use std::any::Any; use std::collections::HashMap; +use std::process::Command; use std::sync::Arc; +use tempdir::TempDir; #[cfg(feature = "azure")] pub mod adls; @@ -21,7 +22,7 @@ pub mod schemas; pub struct TestContext { /// The main table under test pub table: Option, - pub backend: Option>, + pub backend: Option>, /// The configuration used to create the backend. pub config: HashMap, /// An object when it is dropped will clean up any temporary resources created for the test @@ -38,7 +39,7 @@ impl TestContext { let backend_ref = backend.as_ref().map(|s| s.as_str()); let context = match backend_ref { Ok("LOCALFS") | Err(std::env::VarError::NotPresent) => setup_local_context().await, - #[cfg(feature = "azure")] + #[cfg(feature = "azure2")] Ok("AZURE_GEN2") => adls::setup_azure_gen2_context().await, #[cfg(any(feature = "s3", feature = "s3-rustls"))] Ok("S3_LOCAL_STACK") => s3::setup_s3_context().await, @@ -48,7 +49,7 @@ impl TestContext { return context; } - pub fn get_storage(&mut self) -> Arc { + pub fn get_storage(&mut self) -> Arc { if self.backend.is_none() { self.backend = Some(self.new_storage()) } @@ -56,25 +57,25 @@ impl TestContext { return self.backend.as_ref().unwrap().clone(); } - fn new_storage(&self) -> Arc { + fn new_storage(&self) -> Arc { let config = self.config.clone(); let uri = config.get("URI").unwrap().to_string(); - get_backend_for_uri_with_options(&uri, config).unwrap() + DeltaTableBuilder::from_uri(uri) + .with_storage_options(config) + .build_storage() + .unwrap() } pub async fn add_file( &mut self, - path: &str, - data: &[u8], + path: &Path, + data: Bytes, partition_values: &[(&str, Option<&str>)], create_time: i64, commit_to_log: bool, ) { - let uri = self.table.as_ref().unwrap().table_uri.clone(); let backend = self.get_storage(); - let remote_path = uri + "/" + path; - - backend.put_obj(&remote_path, data).await.unwrap(); + backend.put(path, data.clone()).await.unwrap(); if commit_to_log { let mut part_values = HashMap::new(); @@ -83,7 +84,7 @@ impl TestContext { } let add = Add { - path: path.into(), + path: path.as_ref().into(), size: data.len() as i64, modification_time: create_time, partition_values: part_values, @@ -138,7 +139,7 @@ impl TestContext { let backend = self.new_storage(); let p = self.config.get("URI").unwrap().to_string(); - let mut dt = DeltaTable::new(&p, backend, DeltaTableConfig::default()).unwrap(); + let mut dt = DeltaTable::new(backend, DeltaTableConfig::default()); let mut commit_info = Map::::new(); let protocol = action::Protocol { diff --git a/rust/tests/common/s3.rs b/rust/tests/common/s3.rs index 05d1302873..86e76ad8c6 100644 --- a/rust/tests/common/s3.rs +++ b/rust/tests/common/s3.rs @@ -67,7 +67,7 @@ impl S3Cli { ]) .spawn() .expect("aws command is installed"); - child.wait(); + child.wait().unwrap(); } pub fn rm_recurive(&self, prefix: &str, endpoint: &str) { @@ -82,7 +82,7 @@ impl S3Cli { ]) .spawn() .expect("aws command is installed"); - child.wait(); + child.wait().unwrap(); } pub fn delete_table(&self, table_name: &str, endpoint: &str) { @@ -97,7 +97,7 @@ impl S3Cli { ]) .spawn() .expect("aws command is installed"); - child.wait(); + child.wait().unwrap(); } pub fn create_table( @@ -125,7 +125,7 @@ impl S3Cli { ]) .spawn() .expect("aws command is installed"); - child.wait(); + child.wait().unwrap(); } } diff --git a/rust/tests/concurrent_writes_test.rs b/rust/tests/concurrent_writes_test.rs deleted file mode 100644 index b5ab28f99d..0000000000 --- a/rust/tests/concurrent_writes_test.rs +++ /dev/null @@ -1,199 +0,0 @@ -#[cfg(any(feature = "s3", feature = "s3-rustls"))] -#[allow(dead_code)] -mod s3_common; - -#[allow(dead_code)] -mod fs_common; - -use deltalake::{action, DeltaTable}; -use std::collections::HashMap; -use std::future::Future; -use std::iter::FromIterator; -use std::time::Duration; - -#[tokio::test] -#[cfg(any(feature = "s3", feature = "s3-rustls"))] -async fn concurrent_writes_s3() { - s3_common::setup_dynamodb("concurrent_writes"); - s3_common::cleanup_dir_except( - "s3://deltars/concurrent_workers/_delta_log", - vec!["00000000000000000000.json".to_string()], - ) - .await; - run_test(|name| Worker::new("s3://deltars/concurrent_workers", name)).await; -} - -/// An Azure Data Lake Gen2 Storage Account is required to run this test and must be provided by -/// the developer. Because of this requirement, the test cannot run in CI and is therefore marked -/// #[ignore]. As a result, the developer must execute these tests on their machine. -/// In order to execute tests, remove the #[ignore] below and execute via: -/// 'cargo test concurrent_writes_azure --features azure --test concurrent_writes_test -- --nocapture --exact' -/// `AZURE_STORAGE_ACCOUNT_NAME` is required to be set in the environment. -/// `AZURE_STORAGE_ACCOUNT_KEY` is required to be set in the environment. -#[ignore] -#[tokio::test] -#[cfg(feature = "azure")] -async fn concurrent_writes_azure() { - use azure_storage::storage_shared_key_credential::StorageSharedKeyCredential; - use azure_storage_datalake::clients::DataLakeClient; - use chrono::Utc; - use deltalake::DeltaTableConfig; - use deltalake::{DeltaTableMetaData, Schema, SchemaDataType, SchemaField}; - use std::env; - - // Arrange - let storage_account_name = env::var("AZURE_STORAGE_ACCOUNT_NAME").unwrap(); - let storage_account_key = env::var("AZURE_STORAGE_ACCOUNT_KEY").unwrap(); - - let data_lake_client = DataLakeClient::new( - StorageSharedKeyCredential::new( - storage_account_name.to_owned(), - storage_account_key.to_owned(), - ), - None, - ); - - // Create a new file system for test isolation - let file_system_name = format!("test-delta-table-{}", Utc::now().timestamp()); - let file_system_client = data_lake_client.into_file_system_client(file_system_name.to_owned()); - file_system_client.create().into_future().await.unwrap(); - - let table_uri = &format!("adls2://{}/{}/", storage_account_name, file_system_name); - let backend = deltalake::get_backend_for_uri(table_uri).unwrap(); - let mut dt = DeltaTable::new(table_uri, backend, DeltaTableConfig::default()).unwrap(); - - let schema = Schema::new(vec![SchemaField::new( - "Id".to_string(), - SchemaDataType::primitive("integer".to_string()), - true, - HashMap::new(), - )]); - - let metadata = DeltaTableMetaData::new( - Some("Azure Test Table".to_string()), - None, - None, - schema, - vec![], - HashMap::new(), - ); - - let protocol = action::Protocol { - min_reader_version: 1, - min_writer_version: 2, - }; - - dt.create(metadata.clone(), protocol.clone(), None, None) - .await - .unwrap(); - - assert_eq!(0, dt.version()); - assert_eq!(1, dt.get_min_reader_version()); - assert_eq!(2, dt.get_min_writer_version()); - assert_eq!(0, dt.get_files().len()); - assert_eq!(table_uri.trim_end_matches('/').to_string(), dt.table_uri); - - // Act/Assert - run_test(|name| Worker::new(table_uri, name)).await; - - // Cleanup - file_system_client.delete().into_future().await.unwrap(); -} - -#[tokio::test] -async fn concurrent_writes_fs() { - prepare_fs(); - run_test(|name| Worker::new("./tests/data/concurrent_workers", name)).await; -} - -const WORKERS: i64 = 5; -const COMMITS: i64 = 3; - -async fn run_test(create_worker: F) -where - F: Fn(String) -> Fut, - Fut: Future, -{ - let mut workers = Vec::new(); - for w in 0..WORKERS { - workers.push(create_worker(format!("w{}", w)).await); - } - - let mut futures = Vec::new(); - for mut w in workers { - let run = tokio::spawn(async move { w.commit_sequence(COMMITS).await }); - futures.push(run) - } - - let mut map = HashMap::new(); - for f in futures { - map.extend(f.await.unwrap()); - } - - // to ensure that there's been no collisions between workers of acquiring the same version - 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())); - 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())); - files.sort(); - let mut expected = Vec::new(); - for w in 0..WORKERS { - for c in 0..COMMITS { - expected.push(format!("w{}-{}", w, c)) - } - } - assert_eq!(files, expected); -} - -pub struct Worker { - pub table: DeltaTable, - pub name: String, -} - -impl Worker { - pub async fn new(path: &str, name: String) -> Self { - std::env::set_var("DYNAMO_LOCK_OWNER_NAME", &name); - let table = deltalake::open_table(path).await.unwrap(); - Self { table, name } - } - - async fn commit_sequence(&mut self, n: i64) -> HashMap { - let mut result = HashMap::new(); - for i in 0..n { - let name = format!("{}-{}", self.name, i); - let v = self.commit_file(&name).await; - result.insert(v, name); - tokio::time::sleep(Duration::from_millis(100)).await; - } - - result - } - - async fn commit_file(&mut self, name: &str) -> i64 { - let mut tx = self.table.create_transaction(None); - tx.add_action(action::Action::add(action::Add { - path: format!("{}.parquet", name), - size: 396, - partition_values: HashMap::new(), - partition_values_parsed: None, - modification_time: 1564524294000, - data_change: true, - stats: None, - stats_parsed: None, - tags: None, - })); - tx.commit(None, None).await.unwrap() - } -} - -fn prepare_fs() { - fs_common::cleanup_dir_except( - "./tests/data/concurrent_workers/_delta_log", - vec!["00000000000000000000.json".to_string()], - ); -} diff --git a/rust/tests/data/write_exploration/.gitignore b/rust/tests/data/write_exploration/.gitignore deleted file mode 100644 index 12b60fd5f4..0000000000 --- a/rust/tests/data/write_exploration/.gitignore +++ /dev/null @@ -1,4 +0,0 @@ -*.parquet - -_delta_log/*.json -!/_delta_log/00000000000000000000.json diff --git a/rust/tests/data/write_exploration/_delta_log/00000000000000000000.json b/rust/tests/data/write_exploration/_delta_log/00000000000000000000.json deleted file mode 100644 index 7e5c5cffc5..0000000000 --- a/rust/tests/data/write_exploration/_delta_log/00000000000000000000.json +++ /dev/null @@ -1,3 +0,0 @@ -{"commitInfo":{"timestamp":1564524295023,"operation":"CREATE TABLE","operationParameters":{"isManaged":"false","description":null,"partitionBy":"[]","properties":"{}"},"isBlindAppend":true}} -{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} -{"metaData":{"id":"22ef18ba-191c-4c36-a606-3dad5cdf3830","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"modified\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["modified"],"configuration":{},"createdTime":1564524294376}} diff --git a/rust/tests/datafusion_test.rs b/rust/tests/datafusion_test.rs index 25bbfdafea..473d7901d7 100644 --- a/rust/tests/datafusion_test.rs +++ b/rust/tests/datafusion_test.rs @@ -1,339 +1,248 @@ -#[cfg(any(feature = "s3", feature = "s3-rustls"))] -#[allow(dead_code)] -mod s3_common; - -#[cfg(feature = "datafusion-ext")] -mod datafusion { - use std::{collections::HashSet, sync::Arc}; - - use arrow::{ - array::*, - datatypes::{ - DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, - SchemaRef as ArrowSchemaRef, - }, - record_batch::RecordBatch, - }; - use datafusion::datasource::TableProvider; - use datafusion::error::{DataFusionError, Result}; - use datafusion::execution::context::{SessionContext, TaskContext}; - use datafusion::logical_expr::Expr; - use datafusion::logical_plan::Column; - use datafusion::physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, common, file_format::ParquetExec, - metrics::Label, visit_execution_plan, ExecutionPlan, ExecutionPlanVisitor, - }; - use datafusion::scalar::ScalarValue; - use deltalake::{action::SaveMode, operations::DeltaCommands, DeltaTable, DeltaTableMetaData}; - use std::collections::HashMap; - - fn get_scanned_files(node: &dyn ExecutionPlan) -> HashSet