diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index dee9df0aca..9457fe3c3b 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -85,4 +85,4 @@ jobs: run: docker-compose up setup - name: Run tests run: | - cargo test --features s3 + cargo test --features s3,datafusion-ext diff --git a/Cargo.lock b/Cargo.lock index 0da5b3a2a9..15a1f93eaa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -78,16 +78,18 @@ checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" [[package]] name = "arrow" -version = "15.0.0" +version = "18.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6510d919fa4c27880f54430510d09327d7c86699c3692664bc0bb7c314f71385" +checksum = "a5f89d2bc04fa746ee395d20c4cbfa508e4cce5c00bae816f0fae434fcfb9853" dependencies = [ + "ahash", "bitflags", "chrono", "comfy-table", "csv", "flatbuffers", "half", + "hashbrown", "hex", "indexmap", "lazy_static", @@ -371,9 +373,9 @@ checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" [[package]] name = "bytes" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0b3de4a0c5e67e16066a0715723abd91edc2f9001d09c46e1dca929351e130e" +checksum = "ec8a7b6a70fde80372154c65702f00a0f56f3e1c36abbc6c440484be248856db" [[package]] name = "cc" @@ -406,9 +408,9 @@ dependencies = [ [[package]] name = "comfy-table" -version = "5.0.1" +version = "6.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b103d85ca6e209388771bfb7aa6b68a7aeec4afbf6f0a0264bfbf50360e5212e" +checksum = "121d8a5b0346092c18a4b2fd6f620d7a06f0eb7ac0a45860939a0884bc579c56" dependencies = [ "strum", "strum_macros", @@ -471,9 +473,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.5" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c02a4d71819009c192cf4872265391563fd6a84c81ff2c0f2a7026ca4c1d85c" +checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" dependencies = [ "cfg-if", "crossbeam-utils", @@ -481,9 +483,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6455c0ca19f0d2fbf751b908d5c55c1f5cbc65e03c4225427254b46890bdde1e" +checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" dependencies = [ "cfg-if", "crossbeam-epoch", @@ -492,9 +494,9 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.9" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07db9d94cbd326813772c968ccd25999e5f8ae22f4f8d1b11effa37ef6ce281d" +checksum = "045ebe27666471bb549370b4b0b3e51b07f56325befa4284db65fc89c02511b1" dependencies = [ "autocfg", "cfg-if", @@ -506,9 +508,9 @@ dependencies = [ [[package]] name = "crossbeam-queue" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f25d8400f4a7a5778f0e4e52384a48cbd9b5c495d110786187fc750075277a2" +checksum = "1cd42583b04998a5363558e5f9291ee5a5ff6b49944332103f251e7479a82aa7" dependencies = [ "cfg-if", "crossbeam-utils", @@ -516,19 +518,25 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.9" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ff1f980957787286a554052d03c7aee98d99cc32e09f6d45f0a814133c87978" +checksum = "51887d4adc7b564537b15adcfb307936f8075dfcd5f00dde9a9f1d29383682bc" dependencies = [ "cfg-if", "once_cell", ] +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + [[package]] name = "crypto-common" -version = "0.1.3" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57952ca27b5e3606ff4dd79b0020231aaf9d6aa76dc05fd30137538c50bd3ce8" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" dependencies = [ "generic-array", "typenum", @@ -577,9 +585,9 @@ dependencies = [ [[package]] name = "ctor" -version = "0.1.22" +version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f877be4f7c9f246b183111634f75baa039715e3f46ce860677d3b19a69fb229c" +checksum = "cdffe87e1d521a10f9696f833fe502293ea446d7f256c06128293a4119bdf4cb" dependencies = [ "quote", "syn", @@ -587,16 +595,16 @@ dependencies = [ [[package]] name = "datafusion" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f912a89e5ad2f716fcbbad090b1b1bc4b57c07604de1da60925a6652a4b8219" +checksum = "54617e523e447c9a139fdf3682eeca8f909934bd28cdd0032ebd0ff9783775e1" dependencies = [ "ahash", "arrow", "async-trait", + "bytes", "chrono", "datafusion-common", - "datafusion-data-access", "datafusion-expr", "datafusion-optimizer", "datafusion-physical-expr", @@ -609,6 +617,7 @@ dependencies = [ "lazy_static", "log", "num_cpus", + "object_store", "ordered-float 3.0.0", "parking_lot 0.12.1", "parquet", @@ -626,35 +635,22 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec26c175360423abaa97cf45f41c367d07d40f5b631f7772aba4948e1af19e5a" +checksum = "794ca54d3b144038c36b7a31d64c9545abb2edbdda6da055e481fb8a13e4e33b" dependencies = [ "arrow", + "object_store", "ordered-float 3.0.0", "parquet", "sqlparser", ] -[[package]] -name = "datafusion-data-access" -version = "9.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35b28da52ca4e7eb160df15d6119780a7637f3added9e15844c884143d1c8bca" -dependencies = [ - "async-trait", - "chrono", - "futures", - "parking_lot 0.12.1", - "tempfile", - "tokio", -] - [[package]] name = "datafusion-expr" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c4967ba29f27354745154be8d5a03c5236333666b45f3c09e91283021dbb3cf" +checksum = "0087a4e55a861c7040314f217672259304fd26b5f174a065867df6b4ac659896" dependencies = [ "ahash", "arrow", @@ -664,9 +660,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5630b25a6473a58fb096fbbc0b1bf6d28b0b256e5c3d9142a07de270bd3e27b" +checksum = "b822b1a9f4f9c953b142190229085e2856fa9ee52844aa86b40d55edd6e7cc38" dependencies = [ "arrow", "async-trait", @@ -680,9 +676,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca0ed9091539791f406b3928e7802fe65163e4e78dd15d08ad7d67f19c6c6c7d" +checksum = "2328a0e901a89c46391be9445e6e55b6dd8002d4d177e578b0c4a2486ef07cda" dependencies = [ "ahash", "arrow", @@ -705,9 +701,9 @@ dependencies = [ [[package]] name = "datafusion-row" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad857586d0ffd7fbb12b7c9031dcf8801fdbe450b42bf049ef29bb7474c0d4ae" +checksum = "ef6b51e6398ed6dcc5e072c16722b9838f472b0c0ffe25b5df536927cda6044f" dependencies = [ "arrow", "datafusion-common", @@ -717,9 +713,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "9.0.0" +version = "10.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7817f26fbfb3db3310905a83643a99b7518e7f672d1801247d653349268db7b" +checksum = "cb9ae561d6c3dcd09d253ff28f71396b576fca05fe4d0f4fb0e75ee2fc951c72" dependencies = [ "ahash", "arrow", @@ -760,6 +756,7 @@ dependencies = [ "maplit", "num-bigint", "num-traits", + "object_store", "parquet", "parquet-format", "percent-encoding", @@ -783,6 +780,7 @@ dependencies = [ "thiserror", "tokio", "tokio-stream", + "url", "utime", "uuid 1.1.2", "walkdir", @@ -813,9 +811,9 @@ dependencies = [ [[package]] name = "diff" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e25ea47919b1560c4e3b7fe0aaab9becf5b84a10325ddf7db0f0ba5e1026499" +checksum = "56254986775e3233ffa9c4d7d3faaf6d36a2c09d30b20687e9f88bc8bafc16c8" [[package]] name = "digest" @@ -858,11 +856,17 @@ dependencies = [ "winapi", ] +[[package]] +name = "doc-comment" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" + [[package]] name = "dyn-clone" -version = "1.0.6" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "140206b78fb2bc3edbcfc9b5ccbd0b30699cfe8d348b8b31b330e47df5291a5a" +checksum = "9d07a982d1fb29db01e5a59b1918e03da4df7297eaeee7686ac45542fd4e59c8" [[package]] name = "dynamodb_lock" @@ -880,9 +884,9 @@ dependencies = [ [[package]] name = "either" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" +checksum = "3f107b87b6afc2a64fd13cac55fe06d6c8859f12d4b14cbcdd2c67d0976781be" [[package]] name = "encoding_rs" @@ -952,33 +956,33 @@ dependencies = [ [[package]] name = "event-listener" -version = "2.5.2" +version = "2.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" +checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" [[package]] name = "fastrand" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fcf0cee53519c866c09b5de1f6c56ff9d647101f81c1964fa632e148896cdf" +checksum = "a7a407cfaa3385c4ae6b23e84623d48c2798d06e3e6a1878f7f59f17b3f86499" dependencies = [ "instant", ] [[package]] name = "fix-hidden-lifetime-bug" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4b7e739f1d47bbc5185391f0b280ef0bd953fa2327d0bf0c619183502923938" +checksum = "d4ae9c2016a663983d4e40a9ff967d6dcac59819672f0b47f2b17574e99c33c8" dependencies = [ "fix-hidden-lifetime-bug-proc_macros", ] [[package]] name = "fix-hidden-lifetime-bug-proc_macros" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb97774845ef67bfa186b3a39ed2dec87ed70b4932f076ca75392e9c58a6bda1" +checksum = "e4c81935e123ab0741c4c4f0d9b8377e5fb21d3de7e062fa4b1263b1fbcba1ea" dependencies = [ "proc-macro2", "quote", @@ -1198,15 +1202,18 @@ dependencies = [ [[package]] name = "half" -version = "1.8.2" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eabb4a44450da02c90444cf74558da904edde8fb4e9035a9a6a4e15445af0bd7" +checksum = "ad6a9459c9c30b177b925162351f97e7d967c7ea8bab3b8352805327daf45554" +dependencies = [ + "crunchy", +] [[package]] name = "hashbrown" -version = "0.12.1" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db0d4cf898abf0081f964436dc980e96670a0f36863e4b83aaacdb65c9d7ccc3" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ "ahash", ] @@ -1238,12 +1245,9 @@ dependencies = [ [[package]] name = "heck" -version = "0.3.3" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" -dependencies = [ - "unicode-segmentation", -] +checksum = "2540771e65fc8cb83cd6e8a237f70c319bd5c29f78ed1084ba5d50eeac86f7f9" [[package]] name = "hermit-abi" @@ -1496,9 +1500,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.58" +version = "0.3.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fac17f7123a73ca62df411b1bf727ccc805daa070338fda671c86dac1bdc27" +checksum = "258451ab10b34f8af53416d1fdab72c22e805f0c92a1136d59470ec0b11138b2" dependencies = [ "wasm-bindgen", ] @@ -1829,9 +1833,9 @@ dependencies = [ [[package]] name = "num-rational" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d41702bd167c2df5520b384281bc111a4b5efcf7fbc4c9c222c815b07e0a6a6a" +checksum = "0638a1c9d0a3c0914158145bc76cff373a75a627e6ecbfb71cbe6f453a5a19b0" dependencies = [ "autocfg", "num-bigint", @@ -1860,9 +1864,9 @@ dependencies = [ [[package]] name = "oauth2" -version = "4.2.0" +version = "4.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3bd7d544f02ae0fa9e06137962703d043870d7ad6e6d44786d6a5f20679b2c9" +checksum = "6d62c436394991641b970a92e23e8eeb4eb9bca74af4f5badc53bcd568daadbd" dependencies = [ "base64", "chrono", @@ -1878,11 +1882,30 @@ dependencies = [ "url", ] +[[package]] +name = "object_store" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "857af043f5d9f36ed4f71815857f79b841412dda1cf0ca5a29608874f6f038e2" +dependencies = [ + "async-trait", + "bytes", + "chrono", + "futures", + "itertools", + "percent-encoding", + "snafu", + "tokio", + "tracing", + "url", + "walkdir", +] + [[package]] name = "once_cell" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7709cef83f0c1f58f666e746a08b21e0085f7440fa6a29cc194d68aac97a4225" +checksum = "18a6dbe30758c9f83eb00cbea4ac95966305f5a7772f3f42ebfc7fc7eddbd8e1" [[package]] name = "opaque-debug" @@ -2022,9 +2045,9 @@ dependencies = [ [[package]] name = "parquet" -version = "15.0.0" +version = "18.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94d31dde60b151ef88ec2c847e3a8f66d42d7dbdaeefd05d13d79db676b0b56f" +checksum = "65f61759af307fad711e7656c705218402a8a79b776c893c20fef96e8ffd2a7d" dependencies = [ "arrow", "base64", @@ -2033,6 +2056,7 @@ dependencies = [ "bytes", "chrono", "flate2", + "futures", "lz4", "num", "num-bigint", @@ -2040,6 +2064,7 @@ dependencies = [ "rand 0.8.5", "snap", "thrift", + "tokio", "zstd", ] @@ -2066,18 +2091,18 @@ checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" [[package]] name = "pin-project" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58ad3879ad3baf4e44784bc6a718a8698867bb991f8ce24d1bcbe2cfb4c3a75e" +checksum = "78203e83c48cffbe01e4a2d35d566ca4de445d79a85372fc64e378bfc812a260" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744b6f092ba29c3650faf274db506afd39944f48420f6c86b17cfe0ee1cb36bb" +checksum = "710faf75e1b33345361201d36d04e98ac1ed8909151a017ed384700836104c74" dependencies = [ "proc-macro2", "quote", @@ -2156,9 +2181,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.40" +version = "1.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd96a1e8ed2596c337f8eae5f24924ec83f5ad5ab21ea8e455d3566c69fbcaf7" +checksum = "c278e965f1d8cf32d6e0e96de3d3e79712178ae67986d9cf9151f51e95aac89b" dependencies = [ "unicode-ident", ] @@ -2306,9 +2331,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.2.13" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f25bc4c7e55e0b0b7a1d43fb893f4fa1361d0abe38b9ce4f323c2adfe6ef42" +checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" dependencies = [ "bitflags", ] @@ -2609,9 +2634,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.7" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0a5f7c728f5d284929a1cccb5bc19884422bfe6ef4d6c409da2c41838983fcf" +checksum = "24c8ad4f0c00e1eb5bc7614d236a7f1300e3dbd76b68cac8e06fb00b015ad8d8" [[package]] name = "rutie" @@ -2699,9 +2724,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.10" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41d061efea015927ac527063765e73601444cdc344ba855bc7bd44578b25e1c" +checksum = "a2333e6df6d6598f2b1974829f853c2b4c5f4a6e503c10af918081aa6f8564e1" [[package]] name = "serde" @@ -2855,15 +2880,40 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" +checksum = "4614a76b2a8be0058caa9dbbaf66d988527d86d003c11a94fbd335d7661edcef" +dependencies = [ + "autocfg", +] [[package]] name = "smallvec" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2dd574626839106c320a323308629dcb1acfc96e32a8cba364ddc61ac23ee83" +checksum = "2fd0db749597d91ff862fd1d55ea87f7855a744a8425a64695b6fca237d1dad1" + +[[package]] +name = "snafu" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5177903bf45656592d9eb5c0e22f408fc023aae51dbe2088889b71633ba451f2" +dependencies = [ + "doc-comment", + "snafu-derive", +] + +[[package]] +name = "snafu-derive" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "410b26ed97440d90ced3e2488c868d56a86e2064f5d7d6f417909b286afe25e5" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn", +] [[package]] name = "snap" @@ -2910,15 +2960,15 @@ checksum = "9e08d8363704e6c71fc928674353e6b7c23dcea9d82d7012c8faf2a3a025f8d0" [[package]] name = "strum" -version = "0.23.0" +version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cae14b91c7d11c9a851d3fbc80a963198998c2a64eec840477fa92d8ce9b70bb" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" [[package]] name = "strum_macros" -version = "0.23.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bb0dc7ee9c15cea6199cde9a127fa16a4c5819af85395457ad72d68edc85a38" +checksum = "4faebde00e8ff94316c01800f9054fd2ba77d30d9e922541913051d1d978918b" dependencies = [ "heck", "proc-macro2", @@ -3098,9 +3148,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.20.0" +version = "1.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57aec3cfa4c296db7255446efb4928a6be304b431a806216105542a67b6ca82e" +checksum = "7a8325f63a7d4774dd041e363b2409ed1c5cbbd0f867795e661df066b2b0a581" dependencies = [ "autocfg", "bytes", @@ -3193,9 +3243,9 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.35" +version = "0.1.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a400e31aa60b9d44a52a8ee0343b5b18566b03a8321e0d321f695cf56e940160" +checksum = "2fce9567bd60a67d08a16488756721ba392f24f29006402881e43b19aac64307" dependencies = [ "cfg-if", "log", @@ -3206,9 +3256,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6b8ad3567499f98a1db7a752b07a7c8c7c7c34c332ec00effb2b0027974b7c" +checksum = "11c75893af559bc8e10716548bdef5cb2b983f8e637db9d0e15126b61b484ee2" dependencies = [ "proc-macro2", "quote", @@ -3217,9 +3267,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.27" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7709595b8878a4965ce5e87ebf880a7d39c9afc6837721b21a5a816a8117d921" +checksum = "5aeea4303076558a00714b823f9ad67d58a3bbda1df83d8827d21193156e22f7" dependencies = [ "once_cell", "valuable", @@ -3276,15 +3326,15 @@ checksum = "099b7128301d285f79ddd55b9a83d5e6b9e97c92e0ea0daebee7263e932de992" [[package]] name = "unicode-ident" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bd2fe26506023ed7b5e1e315add59d6f584c621d037f9368fea9cfb988f368c" +checksum = "15c61ba63f9235225a22310255a29b806b907c9b8c964bcbd0a2c70f3f2deea7" [[package]] name = "unicode-normalization" -version = "0.1.19" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d54590932941a9e9266f0832deed84ebe1bf2e4c9e4a3554d393d18f5e854bf9" +checksum = "854cbdc4f7bc6ae19c820d44abdc3277ac3e1b2b93db20a636825d9322fb60e6" dependencies = [ "tinyvec", ] @@ -3409,9 +3459,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c53b543413a17a202f4be280a7e5c62a1c69345f5de525ee64f8cfdbc954994" +checksum = "fc7652e3f6c4706c8d9cd54832c4a4ccb9b5336e2c3bd154d5cccfbf1c1f5f7d" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3419,13 +3469,13 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5491a68ab4500fa6b4d726bd67408630c3dbe9c4fe7bda16d5c82a1fd8c7340a" +checksum = "662cd44805586bd52971b9586b1df85cdbbd9112e4ef4d8f41559c334dc6ac3f" dependencies = [ "bumpalo", - "lazy_static", "log", + "once_cell", "proc-macro2", "quote", "syn", @@ -3434,9 +3484,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.31" +version = "0.4.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de9a9cec1733468a8c657e57fa2413d2ae2c0129b95e87c5b72b8ace4d13f31f" +checksum = "fa76fb221a1f8acddf5b54ace85912606980ad661ac7a503b4570ffd3a624dad" dependencies = [ "cfg-if", "js-sys", @@ -3446,9 +3496,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c441e177922bc58f1e12c022624b6216378e5febc2f0533e41ba443d505b80aa" +checksum = "b260f13d3012071dfb1512849c033b1925038373aea48ced3012c09df952c602" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3456,9 +3506,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d94ac45fcf608c1f45ef53e748d35660f168490c10b23704c7779ab8f5c3048" +checksum = "5be8e654bdd9b79216c2929ab90721aa82faf65c48cdf08bdc4e7f51357b80da" dependencies = [ "proc-macro2", "quote", @@ -3469,15 +3519,15 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a89911bd99e5f3659ec4acf9c4d93b0a90fe4a2a11f15328472058edc5261be" +checksum = "6598dd0bd3c7d51095ff6531a5b23e02acdc81804e30d8f07afb77b7215a140a" [[package]] name = "web-sys" -version = "0.3.58" +version = "0.3.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fed94beee57daf8dd7d51f2b15dc2bcde92d7a72304cdf662a4371008b71b90" +checksum = "ed055ab27f941423197eb86b2035720b1a3ce40504df082cac2ecc6ed73335a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -3505,9 +3555,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.22.3" +version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d8de8415c823c8abd270ad483c6feeac771fad964890779f9a8cb24fbbc1bf" +checksum = "f1c760f0d366a6c24a02ed7816e23e691f5d92291f94d15e836006fd11b04daf" dependencies = [ "webpki 0.22.0", ] @@ -3603,9 +3653,9 @@ checksum = "d2d7d3948613f75c98fd9328cfdcc45acc4d360655289d0a7d4ec931392200a3" [[package]] name = "zeroize" -version = "1.5.5" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94693807d016b2f2d2e14420eb3bfcca689311ff775dcf113d74ea624b7cdf07" +checksum = "c394b5bd0c6f669e7275d9c20aa90ae064cb22e75a1cad54e1b34088034b149f" [[package]] name = "zstd" diff --git a/python/src/lib.rs b/python/src/lib.rs index e68486f8e3..a5987c387d 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -488,7 +488,7 @@ impl DeltaStorageFsBackend { } fn normalize_path(&self, path: &str) -> PyResult { - Ok(self._storage.trim_path(path)) + Ok(path.trim_end_matches(std::path::MAIN_SEPARATOR).to_string()) } fn head_obj<'py>(&mut self, py: Python<'py>, path: &str) -> PyResult<&'py PyTuple> { diff --git a/python/tests/test_table_read.py b/python/tests/test_table_read.py index 041be41cbb..edd01f57c6 100644 --- a/python/tests/test_table_read.py +++ b/python/tests/test_table_read.py @@ -1,5 +1,6 @@ import os from datetime import datetime +from pathlib import Path from threading import Barrier, Thread from packaging import version @@ -240,6 +241,9 @@ def test_history_partitioned_table_metadata(): def test_get_files_partitioned_table(): table_path = "../rust/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) + table_path = ( + Path.cwd().parent / "rust/tests/data/delta-0.8.0-partitioned" + ).as_posix() partition_filters = [("day", "=", "3")] assert dt.files_by_partitions(partition_filters=partition_filters) == [ f"{table_path}/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet" diff --git a/python/tests/test_vacuum.py b/python/tests/test_vacuum.py index fbbc116a7c..ef5e84c078 100644 --- a/python/tests/test_vacuum.py +++ b/python/tests/test_vacuum.py @@ -14,10 +14,10 @@ def test_vacuum_dry_run_simple_table(): tombstones = dt.vacuum(retention_periods) tombstones.sort() assert tombstones == [ - "../rust/tests/data/delta-0.2.0/part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet", - "../rust/tests/data/delta-0.2.0/part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet", - "../rust/tests/data/delta-0.2.0/part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet", - "../rust/tests/data/delta-0.2.0/part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet", + "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet", + "part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet", + "part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet", + "part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet", ] retention_periods = -1 @@ -40,16 +40,17 @@ def test_vacuum_zero_duration( ): if use_relative: monkeypatch.chdir(tmp_path) # Make tmp_path the working directory + (tmp_path / "path/to/table").mkdir(parents=True) table_path = "./path/to/table" else: table_path = str(tmp_path) write_deltalake(table_path, sample_data, mode="overwrite") dt = DeltaTable(table_path) - original_files = set(dt.file_uris()) + original_files = set(dt.files()) write_deltalake(table_path, sample_data, mode="overwrite") dt.update_incremental() - new_files = set(dt.file_uris()) + new_files = set(dt.files()) assert new_files.isdisjoint(original_files) tombstones = set(dt.vacuum(retention_hours=0, enforce_retention_duration=False)) @@ -60,10 +61,5 @@ def test_vacuum_zero_duration( ) assert tombstones == original_files - parquet_files = { - os.path.join(table_path, f) - for f in os.listdir(table_path) - if f.endswith("parquet") - } - + parquet_files = {f for f in os.listdir(table_path) if f.endswith("parquet")} assert parquet_files == new_files diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index e0b64fb897..808db8bd1b 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -90,6 +90,7 @@ def test_update_schema(existing_table: DeltaTable): def test_local_path(tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch): monkeypatch.chdir(tmp_path) # Make tmp_path the working directory + (tmp_path / "path/to/table").mkdir(parents=True) local_path = "./path/to/table" write_deltalake(local_path, sample_data) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index b767783c2b..36ad29d7bc 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -26,6 +26,8 @@ lazy_static = "1" percent-encoding = "2" num-bigint = "0.4" num-traits = "0.2.15" +object_store = "0.3.0" +url = "2.2" # HTTP Client reqwest = { version = "0.11", default-features = false, features = [ @@ -67,8 +69,8 @@ async-stream = { version = "0.3.2", default-features = true, optional = true } # High-level writer parquet-format = "~4.0.0" -arrow = "15" -parquet = "15" +arrow = "18" +parquet = "18" crossbeam = { version = "0", optional = true } @@ -81,7 +83,7 @@ walkdir = "2" # rust-dataframe = {version = "0.*", optional = true } [dependencies.datafusion] -version = "9" +version = "10" optional = true [features] diff --git a/rust/src/checkpoints.rs b/rust/src/checkpoints.rs index 0eb8bd589d..9d293b40a9 100644 --- a/rust/src/checkpoints.rs +++ b/rust/src/checkpoints.rs @@ -10,6 +10,7 @@ use chrono::MIN_DATETIME; use futures::StreamExt; use lazy_static::lazy_static; use log::*; +use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; use parquet::arrow::ArrowWriter; use parquet::errors::ParquetError; use regex::Regex; @@ -21,13 +22,13 @@ 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::{ObjectMeta, StorageBackend, StorageError}; use super::table_state::DeltaTableState; use super::time_utils; +use super::DeltaTable; use super::{CheckPoint, DeltaTableError}; -use crate::DeltaTable; /// Error returned when there is an error during creating a checkpoint. #[derive(thiserror::Error, Debug)] @@ -60,12 +61,12 @@ pub enum CheckpointError { #[from] source: ArrowError, }, - /// Passthrough error returned when calling StorageBackend. - #[error("StorageError: {source}")] - Storage { - /// The source StorageError. + /// Passthrough error returned when calling ObjectStore. + #[error("ObjectStoreError: {source}")] + ObjectStore { + /// The source ObjectStoreError. #[from] - source: StorageError, + source: ObjectStoreError, }, /// Passthrough error returned by serde_json. #[error("serde_json::Error: {source}")] @@ -87,13 +88,7 @@ impl From for ArrowError { /// Creates checkpoint at current table version pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), CheckpointError> { - create_checkpoint_for( - table.version(), - table.get_state(), - table.storage.as_ref(), - &table.table_uri, - ) - .await?; + create_checkpoint_for(table.version(), table.get_state(), table.storage.as_ref()).await?; Ok(()) } @@ -107,7 +102,6 @@ pub async fn cleanup_metadata(table: &DeltaTable) -> Result, ) -> Result<(), CheckpointError> { let table = open_table_with_version(table_uri, version).await?; - create_checkpoint_for( - version, - table.get_state(), - table.storage.as_ref(), - table_uri, - ) - .await?; + create_checkpoint_for(version, table.get_state(), table.storage.as_ref()).await?; let enable_expired_log_cleanup = cleanup.unwrap_or_else(|| table.get_state().enable_expired_log_cleanup()); @@ -143,42 +131,37 @@ pub async fn create_checkpoint_from_table_uri_and_cleanup( async fn create_checkpoint_for( version: DeltaDataTypeVersion, state: &DeltaTableState, - storage: &dyn StorageBackend, - table_uri: &str, + storage: &DeltaObjectStore, ) -> Result<(), CheckpointError> { // TODO: checkpoints _can_ be multi-part... haven't actually found a good reference for // an appropriate split point yet though so only writing a single part currently. // See https://github.com/delta-io/delta-rs/issues/288 - - let delta_log_uri = storage.join_path(table_uri, "_delta_log"); - let last_checkpoint_uri = storage.join_path(&delta_log_uri, "_last_checkpoint"); + let last_checkpoint_path = storage.log_path().child("_last_checkpoint"); debug!("Writing parquet bytes to checkpoint buffer."); let parquet_bytes = parquet_bytes_from_state(state)?; - let size = parquet_bytes.len() as i64; - let checkpoint = CheckPoint::new(version, size, None); let file_name = format!("{:020}.checkpoint.parquet", version); - let checkpoint_uri = storage.join_path(&delta_log_uri, &file_name); + let checkpoint_path = storage.log_path().child(file_name); - debug!("Writing checkpoint to {:?}.", checkpoint_uri); - storage.put_obj(&checkpoint_uri, &parquet_bytes).await?; + debug!("Writing checkpoint to {:?}.", checkpoint_path); + storage.put(&checkpoint_path, parquet_bytes).await?; let last_checkpoint_content: Value = serde_json::to_value(&checkpoint)?; - let last_checkpoint_content = serde_json::to_string(&last_checkpoint_content)?; + let last_checkpoint_content = bytes::Bytes::from(serde_json::to_vec(&last_checkpoint_content)?); - debug!("Writing _last_checkpoint to {:?}.", last_checkpoint_uri); + debug!("Writing _last_checkpoint to {:?}.", last_checkpoint_path); storage - .put_obj(&last_checkpoint_uri, last_checkpoint_content.as_bytes()) + .put(&last_checkpoint_path, last_checkpoint_content) .await?; Ok(()) } async fn flush_delete_files bool>( - storage: &dyn StorageBackend, + storage: &DeltaObjectStore, maybe_delete_files: &mut Vec<(DeltaDataTypeVersion, ObjectMeta)>, files_to_delete: &mut Vec<(DeltaDataTypeVersion, ObjectMeta)>, should_delete_file: T, @@ -190,7 +173,7 @@ async fn flush_delete_files bool>( let deleted = files_to_delete .iter_mut() .map(|file| async move { - match storage.delete_obj(&file.1.path).await { + match storage.delete(&file.1.location).await { Ok(_) => Ok(1), Err(e) => Err(DeltaTableError::from(e)), } @@ -212,27 +195,25 @@ async fn flush_delete_files bool>( async fn cleanup_expired_logs_for( until_version: DeltaDataTypeVersion, - storage: &dyn StorageBackend, + storage: &DeltaObjectStore, log_retention_timestamp: i64, - table_uri: &str, ) -> Result { lazy_static! { static ref DELTA_LOG_REGEX: Regex = - Regex::new(r#"^*[/\\]_delta_log[/\\](\d{20})\.(json|checkpoint)*$"#).unwrap(); + Regex::new(r#"_delta_log/(\d{20})\.(json|checkpoint)*$"#).unwrap(); } let mut deleted_log_num = 0; // Get file objects from table. - let log_uri = storage.join_path(table_uri, "_delta_log"); let mut candidates: Vec<(DeltaDataTypeVersion, ObjectMeta)> = Vec::new(); - let mut stream = storage.list_objs(&log_uri).await?; + let mut stream = storage.list(Some(storage.log_path())).await?; while let Some(obj_meta) = stream.next().await { let obj_meta = obj_meta?; - let ts = obj_meta.modified.timestamp_millis(); + let ts = obj_meta.last_modified.timestamp_millis(); - if let Some(captures) = DELTA_LOG_REGEX.captures(&obj_meta.path) { + if let Some(captures) = DELTA_LOG_REGEX.captures(obj_meta.location.as_ref()) { let log_ver_str = captures.get(1).unwrap().as_str(); let log_ver: DeltaDataTypeVersion = log_ver_str.parse().unwrap(); if log_ver < until_version && ts <= log_retention_timestamp { @@ -247,19 +228,19 @@ async fn cleanup_expired_logs_for( let mut last_file: (i64, ObjectMeta) = ( 0, ObjectMeta { - path: String::new(), - modified: MIN_DATETIME, - size: None, + location: Path::from(""), + last_modified: MIN_DATETIME, + size: 0, }, ); let file_needs_time_adjustment = |current_file: &(i64, ObjectMeta), last_file: &(i64, ObjectMeta)| { last_file.0 < current_file.0 - && last_file.1.modified.timestamp() >= current_file.1.modified.timestamp() + && last_file.1.last_modified.timestamp() >= current_file.1.last_modified.timestamp() }; let should_delete_file = |file: &(i64, ObjectMeta)| { - file.1.modified.timestamp() <= log_retention_timestamp && file.0 < until_version + file.1.last_modified.timestamp() <= log_retention_timestamp && file.0 < until_version }; let mut maybe_delete_files: Vec<(DeltaDataTypeVersion, ObjectMeta)> = Vec::new(); @@ -291,9 +272,9 @@ async fn cleanup_expired_logs_for( let updated = ( current_file.0, ObjectMeta { - path: current_file.1.path.clone(), - modified: last_file.1.modified.add(Duration::seconds(1)), - size: None, + location: current_file.1.location.clone(), + last_modified: last_file.1.last_modified.add(Duration::seconds(1)), + size: 0, }, ); maybe_delete_files.push(updated); @@ -320,7 +301,7 @@ async fn cleanup_expired_logs_for( } } -fn parquet_bytes_from_state(state: &DeltaTableState) -> Result, CheckpointError> { +fn parquet_bytes_from_state(state: &DeltaTableState) -> Result { let current_metadata = state .current_metadata() .ok_or(CheckpointError::MissingMetaData)?; @@ -411,7 +392,7 @@ fn parquet_bytes_from_state(state: &DeltaTableState) -> Result, Checkpoi let _ = writer.close()?; debug!("Finished writing checkpoint parquet buffer."); - Ok(bytes) + Ok(bytes::Bytes::from(bytes)) } fn checkpoint_add_from_state( @@ -595,6 +576,7 @@ fn apply_stats_conversion( mod tests { use super::*; use lazy_static::lazy_static; + use std::sync::Arc; use std::time::Duration; use uuid::Uuid; @@ -862,23 +844,38 @@ 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 log_path = |version| format!("{}/_delta_log/{:020}.json", table_path, version); - let backend = crate::storage::get_backend_for_uri(&table_path).unwrap(); + 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 - backend.put_obj(&log_path(0), &[]).await.unwrap(); + 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)); - backend.put_obj(&log_path(1), &[]).await.unwrap(); + object_store + .put(&log_path(1), bytes::Bytes::from("")) + .await + .unwrap(); std::thread::sleep(Duration::from_secs(3)); - backend.put_obj(&log_path(2), &[]).await.unwrap(); + object_store + .put(&log_path(2), bytes::Bytes::from("")) + .await + .unwrap(); - let v0time = backend.head_obj(&log_path(0)).await.unwrap().modified; - let v1time = backend.head_obj(&log_path(1)).await.unwrap().modified; - let v2time = backend.head_obj(&log_path(2)).await.unwrap().modified; + 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 = @@ -890,25 +887,25 @@ mod tests { let removed = crate::checkpoints::cleanup_expired_logs_for( 3, - backend.as_ref(), + object_store.as_ref(), retention_timestamp, - &table_path, ) .await .unwrap(); assert_eq!(removed, 2); - assert!(backend.head_obj(&log_path(0)).await.is_err()); - assert!(backend.head_obj(&log_path(1)).await.is_err()); - assert!(backend.head_obj(&log_path(2)).await.is_ok()); + 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 - backend.delete_obj(&log_path(2)).await.unwrap(); + 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(); } diff --git a/rust/src/delta.rs b/rust/src/delta.rs index cb80028617..8663adc0b4 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -9,6 +9,7 @@ use chrono::{DateTime, Duration, FixedOffset, Utc}; use futures::StreamExt; use lazy_static::lazy_static; use log::*; +use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use parquet::errors::ParquetError; use regex::Regex; use serde::{Deserialize, Serialize}; @@ -29,6 +30,7 @@ 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 @@ -87,6 +89,13 @@ pub enum DeltaTableError { #[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] + source: ObjectStoreError, + }, /// Error returned when reading the checkpoint failed. #[error("Failed to read checkpoint: {}", .source)] ParquetError { @@ -327,7 +336,7 @@ pub enum ApplyLogError { #[error("Failed to read log content")] Storage { /// Storage error details returned while reading the log content. - source: StorageError, + source: ObjectStoreError, }, /// Error returned when reading delta config failed. #[error("Failed to read delta config: {}", .source)] @@ -352,10 +361,10 @@ pub enum ApplyLogError { }, } -impl From for ApplyLogError { - fn from(error: StorageError) -> Self { +impl From for ApplyLogError { + fn from(error: ObjectStoreError) -> Self { match error { - StorageError::NotFound => ApplyLogError::EndOfLog, + ObjectStoreError::NotFound { .. } => ApplyLogError::EndOfLog, _ => ApplyLogError::Storage { source: error }, } } @@ -378,36 +387,19 @@ pub enum LoadCheckpointError { #[error("Failed to read checkpoint content: {source}")] Storage { /// Storage error details returned while reading the checkpoint content. - source: StorageError, + source: ObjectStoreError, }, } -impl From for LoadCheckpointError { - fn from(error: StorageError) -> Self { +impl From for LoadCheckpointError { + fn from(error: ObjectStoreError) -> Self { match error { - StorageError::NotFound => LoadCheckpointError::NotFound, + ObjectStoreError::NotFound { .. } => LoadCheckpointError::NotFound, _ => LoadCheckpointError::Storage { source: error }, } } } -#[inline] -/// Return path relative to parent_path -pub(crate) fn extract_rel_path<'a, 'b>( - parent_path: &'b str, - path: &'a str, -) -> Result<&'a str, DeltaTableError> { - if path.starts_with(&parent_path) { - // plus one to account for path separator - Ok(&path[parent_path.len() + 1..]) - } else { - Err(DeltaTableError::Generic(format!( - "Parent path `{}` is not a prefix of path `{}`", - parent_path, path - ))) - } -} - /// possible version specifications for loading a delta table #[derive(Debug, Clone, PartialEq, Eq)] pub enum DeltaVersion { @@ -554,7 +546,7 @@ impl DeltaTableBuilder { require_files: self.options.require_files, }; - let mut table = DeltaTable::new(&self.options.table_uri, storage, config)?; + let mut table = DeltaTable::new(self.options.table_uri, storage, config)?; match self.options.version { DeltaVersion::Newest => table.load().await?, @@ -587,44 +579,44 @@ pub struct DeltaTable { pub config: DeltaTableConfig, // metadata // application_transactions - pub(crate) storage: Arc, + pub(crate) storage: Arc, last_check_point: Option, - log_uri: String, version_timestamp: HashMap, } impl DeltaTable { - /// Get a shared reference to the underlying storage backend - pub fn get_object_store(&self) -> Arc { + /// get a shared reference to the delta object store + pub fn object_store(&self) -> Arc { self.storage.clone() } + /// Return the uri of commit version. - pub fn commit_uri_from_version(&self, version: DeltaDataTypeVersion) -> String { + pub fn commit_uri_from_version(&self, version: DeltaDataTypeVersion) -> Path { let version = format!("{:020}.json", version); - self.storage.join_path(&self.log_uri, &version) + Path::from_iter(["_delta_log", &version]) } /// Return the list of paths of given checkpoint. - pub fn get_checkpoint_data_paths(&self, check_point: &CheckPoint) -> Vec { - let checkpoint_prefix_pattern = format!("{:020}", check_point.version); - let checkpoint_prefix = self - .storage - .join_path(&self.log_uri, &checkpoint_prefix_pattern); + pub fn get_checkpoint_data_paths(&self, check_point: &CheckPoint) -> Vec { + let checkpoint_prefix = format!("{:020}", check_point.version); + let log_path = self.storage.log_path(); let mut checkpoint_data_paths = Vec::new(); match check_point.parts { None => { - checkpoint_data_paths.push(format!("{}.checkpoint.parquet", checkpoint_prefix)); + let path = log_path.child(&*format!("{}.checkpoint.parquet", checkpoint_prefix)); + checkpoint_data_paths.push(path); } Some(parts) => { for i in 0..parts { - checkpoint_data_paths.push(format!( + let path = log_path.child(&*format!( "{}.checkpoint.{:010}.{:010}.parquet", checkpoint_prefix, i + 1, parts )); + checkpoint_data_paths.push(path); } } } @@ -636,22 +628,20 @@ impl DeltaTable { async fn get_earliest_delta_log_version( &self, ) -> Result { + // TODO check if regex matches against path lazy_static! { static ref DELTA_LOG_REGEX: Regex = - Regex::new(r#"^*[/\\]_delta_log[/\\](\d{20})\.(json|checkpoint)*$"#).unwrap(); + Regex::new(r#"^_delta_log/(\d{20})\.(json|checkpoint)*$"#).unwrap(); } let mut current_delta_log_ver = DeltaDataTypeVersion::MAX; // Get file objects from table. - let log_uri = self - .storage - .join_path(self.table_uri.as_str(), "_delta_log"); - let mut stream = self.storage.list_objs(&log_uri).await?; + let mut stream = self.storage.list(Some(self.storage.log_path())).await?; while let Some(obj_meta) = stream.next().await { let obj_meta = obj_meta?; - if let Some(captures) = DELTA_LOG_REGEX.captures(&obj_meta.path) { + if let Some(captures) = DELTA_LOG_REGEX.captures(obj_meta.location.as_ref()) { let log_ver_str = captures.get(1).unwrap().as_str(); let log_ver: DeltaDataTypeVersion = log_ver_str.parse().unwrap(); if log_ver < current_delta_log_ver { @@ -663,10 +653,10 @@ impl DeltaTable { } async fn get_last_checkpoint(&self) -> Result { - let last_checkpoint_path = self.storage.join_path(&self.log_uri, "_last_checkpoint"); - match self.storage.get_obj(&last_checkpoint_path).await { - Ok(data) => Ok(serde_json::from_slice(&data)?), - Err(StorageError::NotFound) => { + let last_checkpoint_path = Path::from_iter(["_delta_log", "_last_checkpoint"]); + match self.storage.get(&last_checkpoint_path).await { + Ok(data) => Ok(serde_json::from_slice(&data.bytes().await?)?), + Err(ObjectStoreError::NotFound { .. }) => { match self .find_latest_check_point_for_version(DeltaDataTypeVersion::MAX) .await @@ -685,15 +675,14 @@ impl DeltaTable { ) -> Result, DeltaTableError> { lazy_static! { static ref CHECKPOINT_REGEX: Regex = - Regex::new(r#"^*[/\\]_delta_log[/\\](\d{20})\.checkpoint\.parquet$"#).unwrap(); - static ref CHECKPOINT_PARTS_REGEX: Regex = Regex::new( - r#"^*[/\\]_delta_log[/\\](\d{20})\.checkpoint\.\d{10}\.(\d{10})\.parquet$"# - ) - .unwrap(); + Regex::new(r#"^_delta_log/(\d{20})\.checkpoint\.parquet$"#).unwrap(); + static ref CHECKPOINT_PARTS_REGEX: Regex = + Regex::new(r#"^_delta_log/(\d{20})\.checkpoint\.\d{10}\.(\d{10})\.parquet$"#) + .unwrap(); } let mut cp: Option = None; - let mut stream = self.storage.list_objs(&self.log_uri).await?; + let mut stream = self.storage.list(Some(self.storage.log_path())).await?; while let Some(obj_meta) = stream.next().await { // Exit early if any objects can't be listed. @@ -702,10 +691,10 @@ impl DeltaTable { // concurrent writers or if the table is vacuumed by another client. let obj_meta = match obj_meta { Ok(meta) => Ok(meta), - Err(StorageError::NotFound) => continue, + Err(ObjectStoreError::NotFound { .. }) => continue, Err(err) => Err(err), }?; - if let Some(captures) = CHECKPOINT_REGEX.captures(&obj_meta.path) { + if let Some(captures) = CHECKPOINT_REGEX.captures(&obj_meta.location.to_string()) { let curr_ver_str = captures.get(1).unwrap().as_str(); let curr_ver: DeltaDataTypeVersion = curr_ver_str.parse().unwrap(); if curr_ver > version { @@ -722,7 +711,8 @@ impl DeltaTable { continue; } - if let Some(captures) = CHECKPOINT_PARTS_REGEX.captures(&obj_meta.path) { + if let Some(captures) = CHECKPOINT_PARTS_REGEX.captures(&obj_meta.location.to_string()) + { let curr_ver_str = captures.get(1).unwrap().as_str(); let curr_ver: DeltaDataTypeVersion = curr_ver_str.parse().unwrap(); if curr_ver > version { @@ -778,18 +768,18 @@ impl DeltaTable { loop { match self .storage - .head_obj(&self.commit_uri_from_version(version)) + .head(&self.commit_uri_from_version(version)) .await { Ok(meta) => { // also cache timestamp for version self.version_timestamp - .insert(version, meta.modified.timestamp()); + .insert(version, meta.last_modified.timestamp()); version += 1; } Err(e) => { match e { - StorageError::NotFound => { + ObjectStoreError::NotFound { .. } => { version -= 1; if version < 0 { let err = format!( @@ -828,11 +818,12 @@ impl DeltaTable { ) -> Result { let next_version = current_version + 1; let commit_uri = self.commit_uri_from_version(next_version); - let commit_log_bytes = self.storage.get_obj(&commit_uri).await; + let commit_log_bytes = self.storage.get(&commit_uri).await; let commit_log_bytes = match commit_log_bytes { - Err(StorageError::NotFound) => return Ok(PeekCommit::UpToDate), - _ => commit_log_bytes?, - }; + Err(ObjectStoreError::NotFound { .. }) => return Ok(PeekCommit::UpToDate), + Err(err) => Err(err), + Ok(result) => result.bytes().await, + }?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); @@ -908,9 +899,9 @@ impl DeltaTable { ) -> Result<(), DeltaTableError> { // check if version is valid let commit_uri = self.commit_uri_from_version(version); - match self.storage.head_obj(&commit_uri).await { + match self.storage.head(&commit_uri).await { Ok(_) => {} - Err(StorageError::NotFound) => { + Err(ObjectStoreError::NotFound { .. }) => { return Err(DeltaTableError::InvalidVersion(version)); } Err(e) => { @@ -950,9 +941,9 @@ impl DeltaTable { None => { let meta = self .storage - .head_obj(&self.commit_uri_from_version(version)) + .head(&self.commit_uri_from_version(version)) .await?; - let ts = meta.modified.timestamp(); + let ts = meta.last_modified.timestamp(); // also cache timestamp for version self.version_timestamp.insert(version, ts); @@ -1056,25 +1047,11 @@ impl DeltaTable { pub fn get_files_by_partitions( &self, filters: &[PartitionFilter<&str>], - ) -> Result, DeltaTableError> { - let files = self + ) -> Result, DeltaTableError> { + Ok(self .get_active_add_actions_by_partitions(filters)? - .map(|add| add.path.clone()) - .collect(); - - Ok(files) - } - - /// Return the file uris as strings for the partition(s) - #[deprecated( - since = "0.4.0", - note = "Please use the get_file_uris_by_partitions function instead" - )] - pub fn get_file_paths_by_partitions( - &self, - filters: &[PartitionFilter<&str>], - ) -> Result, DeltaTableError> { - self.get_file_uris_by_partitions(filters) + .map(|add| Path::from(add.path.as_ref())) + .collect()) } /// Return the file uris as strings for the partition(s) @@ -1085,51 +1062,40 @@ impl DeltaTable { let files = self.get_files_by_partitions(filters)?; Ok(files .iter() - .map(|fname| self.storage.join_path(&self.table_uri, fname)) + .map(|fname| self.storage.to_uri(fname)) .collect()) } - /// Return a reference to all active "add" actions present in the loaded state - pub fn get_active_add_actions(&self) -> &Vec { - self.state.files() - } - /// Returns an iterator of file names present in the loaded state #[inline] - pub fn get_files_iter(&self) -> impl Iterator { - self.state.files().iter().map(|add| add.path.as_str()) + pub fn get_files_iter(&self) -> impl Iterator + '_ { + self.state + .files() + .iter() + .map(|add| Path::from(add.path.as_ref())) } /// Returns a collection of file names present in the loaded state #[inline] - pub fn get_files(&self) -> Vec<&str> { + pub fn get_files(&self) -> Vec { self.get_files_iter().collect() } /// Returns file names present in the loaded state in HashSet - pub fn get_file_set(&self) -> HashSet<&str> { + pub fn get_file_set(&self) -> HashSet { self.state .files() .iter() - .map(|add| add.path.as_str()) + .map(|add| Path::from(add.path.as_ref())) .collect() } - /// Returns a URIs for all active files present in the current table version. - #[deprecated( - since = "0.4.0", - note = "Please use the get_file_uris function instead" - )] - pub fn get_file_paths(&self) -> Vec { - self.get_file_uris().collect() - } - /// Returns a URIs for all active files present in the current table version. pub fn get_file_uris(&self) -> impl Iterator + '_ { self.state .files() .iter() - .map(|add| self.storage.join_path(&self.table_uri, &add.path)) + .map(|add| self.storage.to_uri(&Path::from(add.path.as_ref()))) } /// Returns statistics for files, in order @@ -1243,10 +1209,12 @@ impl DeltaTable { // move temporary commit file to delta log directory // rely on storage to fail if the file already exists - self.storage - .rename_obj_noreplace(&commit.uri, &self.commit_uri_from_version(version)) + .rename_if_not_exists(&commit.uri, &self.commit_uri_from_version(version)) .await .map_err(|e| match e { - StorageError::AlreadyExists(_) => DeltaTableError::VersionAlreadyExists(version), + ObjectStoreError::AlreadyExists { .. } => { + DeltaTableError::VersionAlreadyExists(version) + } _ => DeltaTableError::from(e), })?; @@ -1260,19 +1228,18 @@ impl DeltaTable { /// 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: &str, + table_uri: impl AsRef, storage_backend: Arc, config: DeltaTableConfig, ) -> Result { - let table_uri = storage_backend.trim_path(table_uri); - let log_uri_normalized = storage_backend.join_path(&table_uri, "_delta_log"); + 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: storage_backend, - table_uri, + storage: Arc::new(storage), + table_uri: root_uri, config, last_check_point: None, - log_uri: log_uri_normalized, version_timestamp: HashMap::new(), }) } @@ -1529,23 +1496,17 @@ impl<'a> DeltaTransaction<'a> { } // Serialize all actions that are part of this log entry. - let log_entry = log_entry_from_actions(&self.actions)?; + let log_entry = bytes::Bytes::from(log_entry_from_actions(&self.actions)?); // Write delta log entry as temporary file to storage. For the actual commit, // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. let token = Uuid::new_v4().to_string(); let file_name = format!("_commit_{}.json.tmp", token); - let uri = self - .delta_table - .storage - .join_path(&self.delta_table.log_uri, &file_name); + let path = Path::from_iter(["_delta_log", &file_name]); - self.delta_table - .storage - .put_obj(&uri, log_entry.as_bytes()) - .await?; + self.delta_table.storage.put(&path, log_entry).await?; - Ok(PreparedCommit { uri }) + Ok(PreparedCommit { uri: path }) } async fn try_commit_loop( @@ -1592,7 +1553,7 @@ impl<'a> DeltaTransaction<'a> { /// Once created, the actual commit could be executed with `DeltaTransaction.try_commit`. #[derive(Debug)] pub struct PreparedCommit { - uri: String, + uri: Path, } fn log_entry_from_actions(actions: &[Action]) -> Result { @@ -1655,7 +1616,7 @@ mod tests { fn normalize_table_uri() { for table_uri in [ "s3://tests/data/delta-0.8.0/", - "s3://tests/data/delta-0.8.0//", + // "s3://tests/data/delta-0.8.0//", "s3://tests/data/delta-0.8.0", ] .iter() @@ -1666,32 +1627,6 @@ mod tests { } } - #[test] - fn rel_path() { - assert!(matches!( - extract_rel_path("data/delta-0.8.0", "data/delta-0.8.0/abc/123"), - Ok("abc/123"), - )); - - assert!(matches!( - extract_rel_path("data/delta-0.8.0", "data/delta-0.8.0/abc.json"), - Ok("abc.json"), - )); - - assert!(matches!( - extract_rel_path("data/delta-0.8.0", "tests/abc.json"), - Err(DeltaTableError::Generic(_)), - )); - - assert!(matches!( - extract_rel_path( - "s3://bucket/database/table/delta-0.8.0", - "s3://bucket/database/table/delta-0.8.0/abc.json" - ), - Ok("abc.json"), - )); - } - #[tokio::test] async fn test_create_delta_table() { // Setup @@ -1726,6 +1661,7 @@ mod tests { let tmp_dir = tempdir::TempDir::new("create_table_test").unwrap(); 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( diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 4d5396831b..9aa91a82f7 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -29,14 +29,15 @@ use async_trait::async_trait; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::{TableProvider, TableType}; +use datafusion::error::Result as DataFusionResult; use datafusion::execution::context::SessionState; use datafusion::logical_plan::Expr; use datafusion::physical_plan::file_format::FileScanConfig; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::{ColumnStatistics, Statistics}; use datafusion::scalar::ScalarValue; +use url::Url; use crate::action; use crate::delta; @@ -46,7 +47,8 @@ impl delta::DeltaTable { /// Return statistics for Datafusion Table pub fn datafusion_table_statistics(&self) -> Statistics { let stats = self - .get_active_add_actions() + .get_state() + .files() .iter() .fold( Some(Statistics { @@ -81,8 +83,8 @@ impl delta::DeltaTable { .get_fields() .iter() .zip(col_stats) - .map(|(field, stats)| ColumnStatistics { - null_count: new_stats + .map(|(field, stats)| { + let null_count = new_stats .null_count .get(field.get_name()) .and_then(|x| { @@ -90,8 +92,9 @@ impl delta::DeltaTable { let null_count = x.as_value()? as usize; Some(null_count_acc + null_count) }) - .or(stats.null_count), - max_value: new_stats + .or(stats.null_count); + + let max_value = new_stats .max_values .get(field.get_name()) .and_then(|x| { @@ -113,8 +116,9 @@ impl delta::DeltaTable { (None, old) => old, } }) - .or_else(|| stats.max_value.clone()), - min_value: new_stats + .or_else(|| stats.max_value.clone()); + + let min_value = new_stats .min_values .get(field.get_name()) .and_then(|x| { @@ -136,8 +140,14 @@ impl delta::DeltaTable { (None, old) => old, } }) - .or_else(|| stats.min_value.clone()), - distinct_count: None, // TODO: distinct + .or_else(|| stats.min_value.clone()); + + ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count: None, // TODO: distinct + } }) .collect() }), @@ -146,6 +156,7 @@ impl delta::DeltaTable { }, ) .unwrap_or_default(); + // Convert column max/min scalar values to correct types based on arrow types. Statistics { is_exact: true, @@ -233,34 +244,42 @@ impl TableProvider for delta::DeltaTable { async fn scan( &self, - _: &SessionState, + session: &SessionState, projection: &Option>, filters: &[Expr], limit: Option, - ) -> datafusion::error::Result> { + ) -> DataFusionResult> { let schema = Arc::new(>::try_from( delta::DeltaTable::schema(self).unwrap(), )?); - let filenames = self.get_file_uris(); - let partitions = filenames - .into_iter() - .zip(self.get_active_add_actions()) - .enumerate() - .map(|(_idx, (fname, action))| { - // TODO: no way to associate stats per file in datafusion at the moment, see: - // https://github.com/apache/arrow-datafusion/issues/1301 - Ok(vec![PartitionedFile::new(fname, action.size as u64)]) - }) - .collect::>()?; + // each delta table must register a specific object store, since paths are internally + // handled relative to the table root. + let object_store_url = self.storage.object_store_url(); + let url: &Url = object_store_url.as_ref(); + session.runtime_env.register_object_store( + url.scheme(), + url.host_str().unwrap_or_default(), + self.object_store(), + ); - let dt_object_store_url = ObjectStoreUrl::parse(&self.table_uri) - .unwrap_or_else(|_| ObjectStoreUrl::local_filesystem()); + // TODO prune files based on file statistics and filter expressions + let partitions = self + .get_state() + .files() + .iter() + .map(|action| { + Ok(vec![PartitionedFile::new( + action.path.clone(), + action.size as u64, + )]) + }) + .collect::>()?; ParquetFormat::default() .create_physical_plan( FileScanConfig { - object_store_url: dt_object_store_url, + object_store_url, file_schema: schema, file_groups: partitions, statistics: self.datafusion_table_statistics(), diff --git a/rust/src/lib.rs b/rust/src/lib.rs index e7a31111be..d501c46d14 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -95,6 +95,7 @@ pub mod data_catalog; mod delta; pub mod delta_arrow; pub mod delta_config; +pub mod object_store; #[cfg(feature = "datafusion-ext")] pub mod operations; pub mod optimize; diff --git a/rust/src/object_store.rs b/rust/src/object_store.rs new file mode 100644 index 0000000000..a7808ecc8f --- /dev/null +++ b/rust/src/object_store.rs @@ -0,0 +1,508 @@ +//! 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/optimize.rs b/rust/src/optimize.rs index 18a339e432..baf963e7e5 100644 --- a/rust/src/optimize.rs +++ b/rust/src/optimize.rs @@ -19,23 +19,22 @@ //! let metrics = Optimize::default().execute(table).await?; //! ```` -use std::collections::HashMap; -use std::sync::Arc; -use std::time::{SystemTime, UNIX_EPOCH}; - -use log::debug; -use log::error; -use parquet::arrow::{ArrowReader, ParquetFileArrowReader}; -use parquet::file::serialized_reader::{SerializedFileReader, SliceableCursor}; - use crate::action::DeltaOperation; use crate::action::{self, Action}; use crate::parquet::file::reader::FileReader; use crate::writer::utils::PartitionPath; use crate::writer::{DeltaWriter, DeltaWriterError, RecordBatchWriter}; use crate::{DeltaDataTypeLong, DeltaTable, DeltaTableError, PartitionFilter}; +use log::debug; +use log::error; +use object_store::{path::Path, ObjectStore}; +use parquet::arrow::{ArrowReader, ParquetFileArrowReader}; +use parquet::file::serialized_reader::SerializedFileReader; use serde::{Deserialize, Serialize}; use serde_json::Map; +use std::collections::HashMap; +use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; /// Metrics from Optimize #[derive(Default, Debug, PartialEq, Clone, Serialize, Deserialize)] @@ -138,7 +137,7 @@ impl From for DeltaOperation { /// A collection of bins for a particular partition #[derive(Debug)] struct MergeBin { - files: Vec, + files: Vec, size_bytes: DeltaDataTypeLong, } @@ -164,7 +163,7 @@ impl MergeBin { self.files.len() } - fn add(&mut self, file_path: String, size: i64) { + fn add(&mut self, file_path: Path, size: i64) { self.files.push(file_path); self.size_bytes += size; } @@ -215,11 +214,8 @@ impl MergePlan { for path in &bin.files { //Read the file into memory and append it to the writer. - - let parquet_uri = table.storage.join_path(&table.table_uri, path); - let data = table.storage.get_obj(&parquet_uri).await?; + let data = table.storage.get(path).await?.bytes().await?; let size: DeltaDataTypeLong = data.len().try_into().unwrap(); - let data = SliceableCursor::new(data); let reader = SerializedFileReader::new(data)?; let records = reader.metadata().file_metadata().num_rows(); @@ -232,7 +228,7 @@ impl MergePlan { writer.write_partition(batch, partition_values).await?; } - actions.push(create_remove(path, partition_values, size)?); + actions.push(create_remove(path.as_ref(), partition_values, size)?); metrics.num_files_removed += 1; metrics.files_removed.total_files += 1; @@ -362,7 +358,7 @@ pub fn create_merge_plan<'a>( } if file.size + curr_bin.get_total_file_size() < target_size { - curr_bin.add(file.path.clone(), file.size); + curr_bin.add(Path::from(file.path.as_str()), file.size); } else { if curr_bin.get_num_files() > 1 { bins.push(curr_bin); @@ -370,7 +366,7 @@ pub fn create_merge_plan<'a>( metrics.total_files_skipped += curr_bin.get_num_files(); } curr_bin = MergeBin::new(); - curr_bin.add(file.path.clone(), file.size); + curr_bin.add(Path::from(file.path.as_str()), file.size); } } diff --git a/rust/src/storage/azure/mod.rs b/rust/src/storage/azure/mod.rs index bf1133f440..99c08218db 100644 --- a/rust/src/storage/azure/mod.rs +++ b/rust/src/storage/azure/mod.rs @@ -16,6 +16,7 @@ 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]. @@ -345,6 +346,23 @@ impl StorageBackend for AdlsGen2Backend { 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, diff --git a/rust/src/storage/file/mod.rs b/rust/src/storage/file/mod.rs index fe537903a0..e33e4ce1a9 100644 --- a/rust/src/storage/file/mod.rs +++ b/rust/src/storage/file/mod.rs @@ -7,9 +7,11 @@ use chrono::DateTime; use futures::{stream::BoxStream, StreamExt}; use std::collections::VecDeque; use std::io; -use std::path::{Path, PathBuf}; +use std::io::SeekFrom; +use std::ops::Range; +use std::path::Path; use tokio::fs; -use tokio::io::AsyncWriteExt; +use tokio::io::{AsyncReadExt, AsyncSeekExt, AsyncWriteExt}; use uuid::Uuid; use walkdir::WalkDir; @@ -44,29 +46,6 @@ impl FileStorageBackend { #[async_trait::async_trait] impl StorageBackend for FileStorageBackend { - #[inline] - fn join_path(&self, path: &str, path_to_join: &str) -> String { - let new_path = Path::new(path); - new_path - .join(path_to_join) - .into_os_string() - .into_string() - .unwrap() - } - - #[inline] - fn join_paths(&self, paths: &[&str]) -> String { - let mut iter = paths.iter(); - let mut path = PathBuf::from(iter.next().unwrap_or(&"")); - iter.for_each(|s| path.push(s)); - path.into_os_string().into_string().unwrap() - } - - #[inline] - fn trim_path(&self, path: &str) -> String { - path.trim_end_matches(std::path::MAIN_SEPARATOR).to_string() - } - async fn head_obj(&self, path: &str) -> Result { let attr = fs::metadata(path).await?; @@ -81,6 +60,29 @@ impl StorageBackend for FileStorageBackend { 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, @@ -274,41 +276,6 @@ mod tests { assert_eq!(fs::metadata(path2).await.is_ok(), false) } - #[test] - fn join_multiple_paths() { - let backend = FileStorageBackend::new("./"); - assert_eq!( - Path::new(&backend.join_paths(&["abc", "efg/", "123"])), - Path::new("abc").join("efg").join("123"), - ); - assert_eq!( - &backend.join_paths(&["abc", "efg"]), - &backend.join_path("abc", "efg"), - ); - assert_eq!(&backend.join_paths(&["foo"]), "foo",); - assert_eq!(&backend.join_paths(&[]), "",); - } - - #[test] - fn trim_path() { - let be = FileStorageBackend::new("root"); - let path = be.join_paths(&["foo", "bar"]); - assert_eq!(be.trim_path(&path), path); - assert_eq!( - be.trim_path(&format!("{}{}", path, std::path::MAIN_SEPARATOR)), - path, - ); - assert_eq!( - be.trim_path(&format!( - "{}{}{}", - path, - std::path::MAIN_SEPARATOR, - std::path::MAIN_SEPARATOR - )), - path, - ); - } - #[test] fn test_parse_uri() { let uri = parse_uri("foo/bar").unwrap(); diff --git a/rust/src/storage/gcs/mod.rs b/rust/src/storage/gcs/mod.rs index d00e73208f..693bd27c66 100644 --- a/rust/src/storage/gcs/mod.rs +++ b/rust/src/storage/gcs/mod.rs @@ -16,6 +16,7 @@ pub(crate) use object::GCSObject; use futures::stream::BoxStream; use std::convert::TryInto; +use std::ops::Range; use log::debug; @@ -65,6 +66,10 @@ impl StorageBackend for GCSStorageBackend { } } + 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, diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index f39f1f4652..97cf1bfff1 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -6,8 +6,10 @@ 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; @@ -427,6 +429,14 @@ pub enum StorageError { /// Uri error details when the URI parsing is invalid. source: InvalidUri, }, + + /// underlying object store returned an error. + #[error("ObjectStore interaction failed: {source}")] + ObjectStore { + /// The wrapped [`ObjectStoreError`] + #[from] + source: ObjectStoreError, + }, } impl StorageError { @@ -493,36 +503,15 @@ impl Clone for ObjectMeta { /// or local storage systems, simply implement this trait. #[async_trait::async_trait] pub trait StorageBackend: Send + Sync + Debug { - /// Create a new path by appending `path_to_join` as a new component to `path`. - #[inline] - fn join_path(&self, path: &str, path_to_join: &str) -> String { - let normalized_path = path.trim_end_matches('/'); - format!("{}/{}", normalized_path, path_to_join) - } - - /// More efficient path join for multiple path components. Use this method if you need to - /// combine more than two path components. - #[inline] - fn join_paths(&self, paths: &[&str]) -> String { - paths - .iter() - .map(|s| s.trim_end_matches('/')) - .collect::>() - .join("/") - } - - /// Returns trimed path with trailing path separator removed. - #[inline] - fn trim_path(&self, path: &str) -> String { - path.trim_end_matches('/').to_string() - } - /// 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, diff --git a/rust/src/storage/s3/mod.rs b/rust/src/storage/s3/mod.rs index 7fad5bb582..3f10caa9f2 100644 --- a/rust/src/storage/s3/mod.rs +++ b/rust/src/storage/s3/mod.rs @@ -3,6 +3,7 @@ use std::collections::HashMap; use std::fmt; use std::fmt::Debug; +use std::ops::Range; use chrono::{DateTime, FixedOffset, Utc}; use futures::stream::BoxStream; @@ -691,6 +692,37 @@ impl StorageBackend for S3StorageBackend { 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?; @@ -908,6 +940,7 @@ async fn get_object_with_retries( bucket: &str, key: &str, retries: usize, + range: Option>, ) -> Result> { let mut tries = 0; loop { @@ -915,6 +948,7 @@ async fn get_object_with_retries( .get_object(GetObjectRequest { bucket: bucket.to_string(), key: key.to_string(), + range: range.as_ref().map(format_http_range), ..Default::default() }) .await; @@ -931,6 +965,10 @@ async fn get_object_with_retries( } } +fn format_http_range(range: &std::ops::Range) -> String { + format!("bytes={}-{}", range.start, range.end.saturating_sub(1)) +} + #[cfg(test)] mod tests { use super::*; @@ -938,23 +976,6 @@ mod tests { use maplit::hashmap; use serial_test::serial; - #[test] - fn join_multiple_paths() { - let backend = S3StorageBackend::new().unwrap(); - assert_eq!(&backend.join_paths(&["abc", "efg/", "123"]), "abc/efg/123",); - assert_eq!(&backend.join_paths(&["abc", "efg/"]), "abc/efg",); - assert_eq!(&backend.join_paths(&["foo"]), "foo",); - assert_eq!(&backend.join_paths(&[]), "",); - } - - #[test] - fn trim_path() { - let be = S3StorageBackend::new().unwrap(); - assert_eq!(be.trim_path("s3://foo/bar"), "s3://foo/bar"); - assert_eq!(be.trim_path("s3://foo/bar/"), "s3://foo/bar"); - assert_eq!(be.trim_path("/foo/bar//"), "/foo/bar"); - } - #[test] fn parse_s3_object_uri() { let uri = parse_uri("s3://foo/bar/baz").unwrap(); diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 93f378d526..f9a7fcf2f4 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -1,22 +1,19 @@ //! The module for delta table state. -use chrono::Utc; -use parquet::file::{ - reader::{FileReader, SerializedFileReader}, - serialized_reader::SliceableCursor, -}; -use serde_json::{Map, Value}; -use std::collections::HashMap; -use std::collections::HashSet; -use std::convert::TryFrom; -use std::io::{BufRead, BufReader, Cursor}; - use super::{ ApplyLogError, CheckPoint, DeltaDataTypeLong, DeltaDataTypeVersion, DeltaTable, DeltaTableError, DeltaTableMetaData, }; use crate::action::{self, Action}; use crate::delta_config; +use chrono::Utc; +use object_store::ObjectStore; +use parquet::file::reader::{FileReader, SerializedFileReader}; +use serde_json::{Map, Value}; +use std::collections::HashMap; +use std::collections::HashSet; +use std::convert::TryFrom; +use std::io::{BufRead, BufReader, Cursor}; /// State snapshot currently held by the Delta Table instance. #[derive(Default, Debug, Clone)] @@ -49,7 +46,7 @@ impl DeltaTableState { version: DeltaDataTypeVersion, ) -> Result { let commit_uri = table.commit_uri_from_version(version); - let commit_log_bytes = table.storage.get_obj(&commit_uri).await?; + let commit_log_bytes = table.storage.get(&commit_uri).await?.bytes().await?; let reader = BufReader::new(Cursor::new(commit_log_bytes)); let mut new_state = DeltaTableState::with_version(version); @@ -87,8 +84,8 @@ impl DeltaTableState { let mut new_state = DeltaTableState::with_version(check_point.version); for f in &checkpoint_data_paths { - let obj = table.storage.get_obj(f).await?; - let preader = SerializedFileReader::new(SliceableCursor::new(obj))?; + let obj = table.storage.get(f).await?.bytes().await?; + let preader = SerializedFileReader::new(obj)?; let schema = preader.metadata().file_metadata().schema(); if !schema.is_group() { return Err(DeltaTableError::from(action::ActionError::Generic( diff --git a/rust/src/vacuum.rs b/rust/src/vacuum.rs index f837bb30b9..f1133f8483 100644 --- a/rust/src/vacuum.rs +++ b/rust/src/vacuum.rs @@ -21,10 +21,10 @@ //! let metrics = Vacuum::default().execute(table).await?; //! ```` -use crate::delta::extract_rel_path; use crate::{DeltaDataTypeLong, DeltaTable, DeltaTableError}; use chrono::{Duration, Utc}; use futures::StreamExt; +use object_store::{path::Path, ObjectStore}; use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; @@ -57,7 +57,7 @@ impl Default for Vacuum { /// Encapsulate which files are to be deleted and the parameters used to make that decision pub struct VacuumPlan { /// What files are to be deleted - pub files_to_delete: Vec, + pub files_to_delete: Vec, } /// Details for the Vacuum operation including which files were @@ -115,8 +115,9 @@ pub(crate) fn get_stale_files( /// indexes and these must be deleted when the data they are tied to is deleted. pub(crate) fn is_hidden_directory( table: &DeltaTable, - path_name: &str, + path: &Path, ) -> Result { + let path_name = path.to_string(); Ok((path_name.starts_with('.') || path_name.starts_with('_')) && !path_name.starts_with("_delta_index") && !path_name.starts_with("_change_data") @@ -140,12 +141,15 @@ impl VacuumPlan { } // Delete the files - let files_deleted = match table.storage.delete_objs(&self.files_to_delete).await { + let files_deleted = match table.storage.delete_batch(&self.files_to_delete).await { Ok(_) => Ok(self.files_to_delete), - Err(err) => Err(VacuumError::from(DeltaTableError::StorageError { + Err(err) => Err(VacuumError::from(DeltaTableError::ObjectStore { source: err, })), - }?; + }? + .into_iter() + .map(|file| file.to_string()) + .collect(); Ok(VacuumMetrics { files_deleted, @@ -181,22 +185,21 @@ pub async fn create_vacuum_plan( let mut files_to_delete = vec![]; let mut all_files = table .storage - .list_objs(&table.table_uri) + .list(None) .await .map_err(DeltaTableError::from)?; while let Some(obj_meta) = all_files.next().await { + // TODO should we allow NotFound here in case we have a temporary commit file in the list let obj_meta = obj_meta.map_err(DeltaTableError::from)?; - let rel_path = extract_rel_path(&table.table_uri, &obj_meta.path)?; - - if valid_files.contains(rel_path) // file is still being tracked in table - || !expired_tombstones.contains(rel_path) // file is not an expired tombstone - || is_hidden_directory(table, rel_path)? + if valid_files.contains(&obj_meta.location) // file is still being tracked in table + || !expired_tombstones.contains(obj_meta.location.as_ref()) // file is not an expired tombstone + || is_hidden_directory(table, &obj_meta.location)? { continue; } - files_to_delete.push(obj_meta.path); + files_to_delete.push(obj_meta.location); } Ok(VacuumPlan { files_to_delete }) @@ -228,7 +231,7 @@ impl Vacuum { let plan = create_vacuum_plan(table, self).await?; if dry_run { return Ok(VacuumMetrics { - files_deleted: plan.files_to_delete, + files_deleted: plan.files_to_delete.iter().map(|f| f.to_string()).collect(), dry_run: true, }); } diff --git a/rust/src/writer/json.rs b/rust/src/writer/json.rs index 5e72e2e62e..2bbcfe86b8 100644 --- a/rust/src/writer/json.rs +++ b/rust/src/writer/json.rs @@ -7,16 +7,15 @@ use super::{ }, DeltaWriter, DeltaWriterError, }; -use crate::writer::utils::ShareableBuffer; -use crate::{ - action::Add, get_backend_for_uri_with_options, DeltaTable, DeltaTableMetaData, Schema, - StorageBackend, -}; +use crate::{action::Add, DeltaTable, DeltaTableMetaData, Schema}; +use crate::{object_store::DeltaObjectStore, writer::utils::ShareableBuffer}; use arrow::{ datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}, record_batch::*, }; +use bytes::Bytes; use log::{info, warn}; +use object_store::ObjectStore; use parquet::{ arrow::ArrowWriter, basic::Compression, errors::ParquetError, file::properties::WriterProperties, @@ -30,8 +29,7 @@ type BadValue = (Value, ParquetError); /// Writes messages to a delta lake table. pub struct JsonWriter { - storage: Arc, - table_uri: String, + storage: Arc, arrow_schema_ref: Arc, writer_properties: WriterProperties, partition_columns: Vec, @@ -186,8 +184,7 @@ impl JsonWriter { partition_columns: Option>, storage_options: Option>, ) -> Result { - let storage = - get_backend_for_uri_with_options(&table_uri, storage_options.unwrap_or_default())?; + let storage = DeltaObjectStore::try_new_with_options(&table_uri, storage_options)?; // Initialize writer properties for the underlying arrow writer let writer_properties = WriterProperties::builder() @@ -196,8 +193,7 @@ impl JsonWriter { .build(); Ok(Self { - storage, - table_uri, + storage: Arc::new(storage), arrow_schema_ref: schema, writer_properties, partition_columns: partition_columns.unwrap_or_default(), @@ -221,7 +217,6 @@ impl JsonWriter { Ok(Self { storage: table.storage.clone(), - table_uri: table.table_uri.clone(), arrow_schema_ref, writer_properties, partition_columns, @@ -364,21 +359,10 @@ impl DeltaWriter> for JsonWriter { for (_, mut writer) in writers { let metadata = writer.arrow_writer.close()?; - let path = next_data_path(&self.partition_columns, &writer.partition_values, None)?; - - let obj_bytes = writer.buffer.to_vec(); + let obj_bytes = Bytes::from(writer.buffer.to_vec()); let file_size = obj_bytes.len() as i64; - - let storage_path = self.storage.join_path(&self.table_uri, path.as_str()); - - // - // TODO: Wrap in retry loop to handle temporary network errors - // - - self.storage - .put_obj(&storage_path, obj_bytes.as_slice()) - .await?; + self.storage.put(&path, obj_bytes).await?; // Replace self null_counts with an empty map. Use the other for stats. let null_counts = std::mem::take(&mut writer.null_counts); @@ -386,7 +370,7 @@ impl DeltaWriter> for JsonWriter { actions.push(create_add( &writer.partition_values, null_counts, - path, + path.to_string(), file_size, &metadata, )?); diff --git a/rust/src/writer/mod.rs b/rust/src/writer/mod.rs index eea9fea895..3c036db692 100644 --- a/rust/src/writer/mod.rs +++ b/rust/src/writer/mod.rs @@ -17,6 +17,7 @@ use crate::{ use arrow::{datatypes::SchemaRef, datatypes::*, error::ArrowError}; use async_trait::async_trait; pub use json::JsonWriter; +use object_store::Error as ObjectStoreError; use parquet::{basic::LogicalType, errors::ParquetError}; pub use record_batch::RecordBatchWriter; use serde_json::Value; @@ -79,6 +80,14 @@ pub enum DeltaWriterError { source: StorageError, }, + /// underlying object store returned an error. + #[error("ObjectStore interaction failed: {source}")] + ObjectStore { + /// The wrapped [`ObjectStoreError`] + #[from] + source: ObjectStoreError, + }, + /// Arrow returned an error. #[error("Arrow interaction failed: {source}")] Arrow { diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index aa9f295d03..495d070861 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -36,10 +36,7 @@ use super::{ }; use crate::writer::stats::apply_null_counts; use crate::writer::utils::ShareableBuffer; -use crate::{ - action::Add, get_backend_for_uri_with_options, DeltaTable, DeltaTableMetaData, Schema, - StorageBackend, -}; +use crate::{action::Add, object_store::DeltaObjectStore, DeltaTable, DeltaTableMetaData, Schema}; use arrow::record_batch::RecordBatch; use arrow::{ array::{Array, UInt32Array}, @@ -47,6 +44,8 @@ use arrow::{ datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}, error::ArrowError, }; +use bytes::Bytes; +use object_store::ObjectStore; use parquet::{arrow::ArrowWriter, errors::ParquetError}; use parquet::{basic::Compression, file::properties::WriterProperties}; use std::collections::HashMap; @@ -55,8 +54,7 @@ use std::sync::Arc; /// Writes messages to a delta lake table. pub struct RecordBatchWriter { - pub(crate) storage: Arc, - pub(crate) table_uri: String, + pub(crate) storage: Arc, pub(crate) arrow_schema_ref: Arc, pub(crate) writer_properties: WriterProperties, pub(crate) partition_columns: Vec, @@ -77,8 +75,7 @@ impl RecordBatchWriter { partition_columns: Option>, storage_options: Option>, ) -> Result { - let storage = - get_backend_for_uri_with_options(&table_uri, storage_options.unwrap_or_default())?; + let storage = DeltaObjectStore::try_new_with_options(&table_uri, storage_options)?; // Initialize writer properties for the underlying arrow writer let writer_properties = WriterProperties::builder() @@ -87,8 +84,7 @@ impl RecordBatchWriter { .build(); Ok(Self { - storage, - table_uri: table_uri.clone(), + storage: Arc::new(storage), arrow_schema_ref: schema, writer_properties, partition_columns: partition_columns.unwrap_or_default(), @@ -112,7 +108,6 @@ impl RecordBatchWriter { Ok(Self { storage: table.storage.clone(), - table_uri: table.table_uri.clone(), arrow_schema_ref, writer_properties, partition_columns, @@ -231,22 +226,10 @@ impl DeltaWriter for RecordBatchWriter { for (_, mut writer) in writers { let metadata = writer.arrow_writer.close()?; - let path = next_data_path(&self.partition_columns, &writer.partition_values, None)?; - - let obj_bytes = writer.buffer.to_vec(); + let obj_bytes = Bytes::from(writer.buffer.to_vec()); let file_size = obj_bytes.len() as i64; - let storage_path = self - .storage - .join_path(self.table_uri.as_str(), path.as_str()); - - // - // TODO: Wrap in retry loop to handle temporary network errors - // - - self.storage - .put_obj(&storage_path, obj_bytes.as_slice()) - .await?; + self.storage.put(&path, obj_bytes).await?; // Replace self null_counts with an empty map. Use the other for stats. let null_counts = std::mem::take(&mut writer.null_counts); @@ -254,7 +237,7 @@ impl DeltaWriter for RecordBatchWriter { actions.push(create_add( &writer.partition_values, null_counts, - path, + path.to_string(), file_size, &metadata, )?); diff --git a/rust/src/writer/utils.rs b/rust/src/writer/utils.rs index 83a41946d4..fd3adb430c 100644 --- a/rust/src/writer/utils.rs +++ b/rust/src/writer/utils.rs @@ -9,6 +9,7 @@ use arrow::{ json::reader::{Decoder, DecoderOptions}, record_batch::*, }; +use object_store::path::Path; use serde_json::Value; use std::collections::HashMap; use std::fmt::Display; @@ -72,7 +73,7 @@ pub(crate) fn next_data_path( partition_columns: &[String], partition_values: &HashMap>, part: Option, -) -> Result { +) -> Result { // TODO: what does 00000 mean? // TODO (roeap): my understanding is, that the values are used as a counter - i.e. if a single batch of // data written to one partition needs to be split due to desired file size constraints. @@ -91,11 +92,11 @@ pub(crate) fn next_data_path( ); if partition_columns.is_empty() { - return Ok(file_name); + return Ok(Path::from(file_name)); } let partition_key = PartitionPath::from_hashmap(partition_columns, partition_values)?; - Ok(format!("{}/{}", partition_key, file_name)) + Ok(Path::from(format!("{}/{}", partition_key, file_name))) } /// partition json values diff --git a/rust/tests/adls_gen2_table_test.rs b/rust/tests/adls_gen2_table_test.rs index 531ce0b83d..dd41835bb0 100644 --- a/rust/tests/adls_gen2_table_test.rs +++ b/rust/tests/adls_gen2_table_test.rs @@ -15,6 +15,7 @@ mod adls_gen2_table { action, DeltaTable, DeltaTableConfig, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, }; + use object_store::path::Path; use serial_test::serial; use std::collections::HashMap; use std::env; @@ -39,11 +40,11 @@ mod adls_gen2_table { assert_eq!( table.get_files(), vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", + 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"), ] ); @@ -93,11 +94,11 @@ mod adls_gen2_table { assert_eq!( table.get_files(), vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", + 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"), ] ); diff --git a/rust/tests/checkpoint_writer_test.rs b/rust/tests/checkpoint_writer_test.rs index a8cb108d8a..737af3939d 100644 --- a/rust/tests/checkpoint_writer_test.rs +++ b/rust/tests/checkpoint_writer_test.rs @@ -1,15 +1,14 @@ +use ::object_store::path::Path as ObjectStorePath; +use chrono::Utc; +use deltalake::action::*; +use deltalake::*; +use maplit::hashmap; use std::collections::HashSet; use std::fs; use std::iter::FromIterator; use std::path::{Path, PathBuf}; - -use chrono::Utc; -use maplit::hashmap; use uuid::Uuid; -use deltalake::action::*; -use deltalake::*; - #[allow(dead_code)] mod fs_common; @@ -140,8 +139,15 @@ mod delete_expired_delta_log_in_checkpoint { ) .await .unwrap(); + table.update().await.unwrap(); // make table to read the checkpoint - assert_eq!(table.get_files(), vec![a1.path.as_str(), a2.path.as_str()]); + assert_eq!( + table.get_files(), + vec![ + ObjectStorePath::from(a1.path.as_ref()), + ObjectStorePath::from(a2.path.as_ref()) + ] + ); // log files 0 and 1 are deleted table @@ -152,7 +158,6 @@ mod delete_expired_delta_log_in_checkpoint { .load_version(1) .await .expect_err("Should not load version 1"); - // log file 2 is kept table.load_version(2).await.expect("Cannot load version 2"); } @@ -184,7 +189,13 @@ mod delete_expired_delta_log_in_checkpoint { .await .unwrap(); table.update().await.unwrap(); // make table to read the checkpoint - assert_eq!(table.get_files(), vec![a1.path.as_str(), a2.path.as_str()]); + assert_eq!( + table.get_files(), + vec![ + ObjectStorePath::from(a1.path.as_ref()), + ObjectStorePath::from(a2.path.as_ref()) + ] + ); table .load_version(0) @@ -215,15 +226,27 @@ mod checkpoints_with_tombstones { assert_eq!(2, fs_common::commit_add(&mut table, &a2).await); checkpoints::create_checkpoint(&table).await.unwrap(); table.update().await.unwrap(); // make table to read the checkpoint - assert_eq!(table.get_files(), vec![a1.path.as_str(), a2.path.as_str()]); + assert_eq!( + table.get_files(), + vec![ + ObjectStorePath::from(a1.path.as_ref()), + ObjectStorePath::from(a2.path.as_ref()) + ] + ); let (removes1, opt1) = pseudo_optimize(&mut table, 5 * 59 * 1000).await; - assert_eq!(table.get_files(), vec![opt1.path.as_str()]); + assert_eq!( + table.get_files(), + vec![ObjectStorePath::from(opt1.path.as_ref())] + ); assert_eq!(table.get_state().all_tombstones(), &removes1); checkpoints::create_checkpoint(&table).await.unwrap(); table.update().await.unwrap(); // make table to read the checkpoint - assert_eq!(table.get_files(), vec![opt1.path.as_str()]); + assert_eq!( + table.get_files(), + vec![ObjectStorePath::from(opt1.path.as_ref())] + ); assert_eq!(table.get_state().all_tombstones().len(), 0); // stale removes are deleted from the state } diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 624246ff22..583cb172a2 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -70,7 +70,7 @@ impl TestContext { create_time: i64, commit_to_log: bool, ) { - let uri = self.table.as_ref().unwrap().table_uri.to_string(); + let uri = self.table.as_ref().unwrap().table_uri.clone(); let backend = self.get_storage(); let remote_path = uri + "/" + path; diff --git a/rust/tests/datafusion_test.rs b/rust/tests/datafusion_test.rs index 1c9c5dd6ea..db3ca92e51 100644 --- a/rust/tests/datafusion_test.rs +++ b/rust/tests/datafusion_test.rs @@ -1,3 +1,7 @@ +#[cfg(feature = "s3")] +#[allow(dead_code)] +mod s3_common; + #[cfg(feature = "datafusion-ext")] mod datafusion { use std::sync::Arc; @@ -121,4 +125,58 @@ mod datafusion { Ok(()) } + + #[cfg(feature = "s3")] + mod s3 { + use super::*; + use crate::s3_common::setup; + use deltalake::s3_storage_options; + use deltalake::storage; + use dynamodb_lock::dynamo_lock_options; + use maplit::hashmap; + use serial_test::serial; + + #[tokio::test] + #[serial] + async fn test_datafusion_simple_query() -> Result<()> { + setup(); + + // Use the manual options API so we have some basic integrationcoverage. + let table_uri = "s3://deltars/simple"; + let storage = storage::get_backend_for_uri_with_options( + table_uri, + hashmap! { + s3_storage_options::AWS_REGION.to_string() => "us-east-2".to_string(), + dynamo_lock_options::DYNAMO_LOCK_OWNER_NAME.to_string() => "s3::deltars/simple".to_string(), + }, + ) + .unwrap(); + let mut table = deltalake::DeltaTable::new( + table_uri, + storage, + deltalake::DeltaTableConfig::default(), + ) + .unwrap(); + table.load().await.unwrap(); + + let ctx = SessionContext::new(); + ctx.register_table("demo", Arc::new(table))?; + + let batches = ctx + .sql("SELECT id FROM demo WHERE id > 5 ORDER BY id ASC") + .await? + .collect() + .await?; + + assert_eq!(batches.len(), 1); + let batch = &batches[0]; + + assert_eq!( + batch.column(0).as_ref(), + Arc::new(Int64Array::from(vec![7, 9])).as_ref(), + ); + + Ok(()) + } + } } diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index d6335d7589..bf2cefd974 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -34,6 +34,7 @@ pub async fn create_table_from_json( assert!(path.starts_with("./tests/data")); std::fs::create_dir_all(path).unwrap(); std::fs::remove_dir_all(path).unwrap(); + std::fs::create_dir_all(path).unwrap(); let schema: Schema = serde_json::from_value(schema).unwrap(); let config: HashMap> = serde_json::from_value(config).unwrap(); create_test_table(path, schema, partition_columns, config).await diff --git a/rust/tests/gcs_test.rs b/rust/tests/gcs_test.rs index a01a83b33b..43ab1e19d7 100644 --- a/rust/tests/gcs_test.rs +++ b/rust/tests/gcs_test.rs @@ -1,5 +1,6 @@ #[cfg(feature = "gcs")] mod gcs { + use object_store::path::Path; /* * The storage account to run this test must be provided by the developer and test are executed locally. * @@ -24,11 +25,11 @@ mod gcs { assert_eq!( table.get_files(), vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", + 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(); diff --git a/rust/tests/optimize_test.rs b/rust/tests/optimize_test.rs index 373cd6c853..d87c1b77a7 100644 --- a/rust/tests/optimize_test.rs +++ b/rust/tests/optimize_test.rs @@ -197,7 +197,7 @@ mod optimize { .await?; let version = dt.version(); - assert_eq!(dt.get_active_add_actions().len(), 5); + assert_eq!(dt.get_state().files().len(), 5); let optimize = Optimize::default().target_size(2_000_000); let metrics = optimize.execute(&mut dt).await?; @@ -207,7 +207,7 @@ mod optimize { assert_eq!(metrics.num_files_removed, 4); assert_eq!(metrics.total_considered_files, 5); assert_eq!(metrics.partitions_optimized, 1); - assert_eq!(dt.get_active_add_actions().len(), 2); + assert_eq!(dt.get_state().files().len(), 2); Ok(()) } @@ -263,7 +263,7 @@ mod optimize { assert_eq!(version + 1, dt.version()); assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); - assert_eq!(dt.get_active_add_actions().len(), 3); + assert_eq!(dt.get_state().files().len(), 3); Ok(()) } @@ -298,7 +298,7 @@ mod optimize { let uri = context.tmp_dir.path().to_str().to_owned().unwrap(); let mut other_dt = deltalake::open_table(uri).await?; - let add = &other_dt.get_active_add_actions()[0]; + let add = &other_dt.get_state().files()[0]; let remove = Remove { path: add.path.clone(), deletion_timestamp: Some( diff --git a/rust/tests/read_delta_partitions_test.rs b/rust/tests/read_delta_partitions_test.rs index f2d2fae679..7312f70661 100644 --- a/rust/tests/read_delta_partitions_test.rs +++ b/rust/tests/read_delta_partitions_test.rs @@ -138,8 +138,6 @@ async fn read_null_partitions_from_checkpoint() { ) .await; - println!("{}", table.table_uri); - let delta_log = std::path::Path::new(&table.table_uri).join("_delta_log"); let add = |partition: Option| Add { diff --git a/rust/tests/read_delta_test.rs b/rust/tests/read_delta_test.rs index fe66de619b..4b09a2149b 100644 --- a/rust/tests/read_delta_test.rs +++ b/rust/tests/read_delta_test.rs @@ -3,6 +3,7 @@ extern crate deltalake; use chrono::Utc; use deltalake::DeltaTableBuilder; use deltalake::PeekCommit; +use object_store::path::Path; use pretty_assertions::assert_eq; use std::collections::HashMap; @@ -20,9 +21,9 @@ async fn read_delta_2_0_table_without_version() { assert_eq!( table.get_files(), vec![ - "part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet", - "part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet", - "part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet", + Path::from("part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet"), + Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), + Path::from("part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet"), ] ); let tombstones = table.get_state().all_tombstones(); @@ -68,8 +69,8 @@ async fn read_delta_table_ignoring_tombstones() { assert_eq!( table.get_files(), vec![ - "part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet", - "part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet" + Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), + Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet") ] ); } @@ -127,8 +128,8 @@ async fn read_delta_2_0_table_with_version() { assert_eq!( table.get_files(), vec![ - "part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet", - "part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet", + Path::from("part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet"), + Path::from("part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet"), ], ); @@ -141,8 +142,8 @@ async fn read_delta_2_0_table_with_version() { assert_eq!( table.get_files(), vec![ - "part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet", - "part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet", + Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), + Path::from("part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet"), ] ); @@ -155,9 +156,9 @@ async fn read_delta_2_0_table_with_version() { assert_eq!( table.get_files(), vec![ - "part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet", - "part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet", - "part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet", + Path::from("part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet"), + Path::from("part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet"), + Path::from("part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet"), ] ); } @@ -173,8 +174,8 @@ async fn read_delta_8_0_table_without_version() { assert_eq!( table.get_files(), vec![ - "part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet", - "part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet" + Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), + Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet") ] ); assert_eq!(table.get_stats().count(), 2); @@ -218,8 +219,8 @@ async fn read_delta_8_0_table_with_load_version() { assert_eq!( table.get_files(), vec![ - "part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet", - "part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet", + Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), + Path::from("part-00000-04ec9591-0b73-459e-8d18-ba5711d6cbe1-c000.snappy.parquet"), ] ); table.load_version(0).await.unwrap(); @@ -229,14 +230,15 @@ async fn read_delta_8_0_table_with_load_version() { assert_eq!( table.get_files(), vec![ - "part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet", - "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet", + Path::from("part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a1-c000.snappy.parquet"), + Path::from("part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"), ] ); } #[tokio::test] async fn read_delta_8_0_table_with_partitions() { + let current_dir = Path::from_filesystem_path(std::env::current_dir().unwrap()).unwrap(); let table = deltalake::open_table("./tests/data/delta-0.8.0-partitioned") .await .unwrap(); @@ -255,8 +257,8 @@ async fn read_delta_8_0_table_with_partitions() { assert_eq!( table.get_files_by_partitions(&filters).unwrap(), vec![ - "year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet".to_string(), - "year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet".to_string() + Path::from("year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet"), + Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet") ] ); @@ -264,16 +266,16 @@ async fn read_delta_8_0_table_with_partitions() { assert_eq!( table.get_file_uris_by_partitions(&filters).unwrap(), vec![ - "./tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet".to_string(), - "./tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet".to_string() + format!("/{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet", current_dir.as_ref()), + format!("/{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet", current_dir.as_ref()) ] ); #[cfg(windows)] assert_eq!( table.get_file_uris_by_partitions(&filters).unwrap(), vec![ - "./tests/data/delta-0.8.0-partitioned\\year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet".to_string(), - "./tests/data/delta-0.8.0-partitioned\\year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet".to_string() + format!("{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet", current_dir.as_ref()), + format!("{}/tests/data/delta-0.8.0-partitioned/year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet", current_dir.as_ref()) ] ); @@ -284,10 +286,10 @@ async fn read_delta_8_0_table_with_partitions() { assert_eq!( table.get_files_by_partitions(&filters).unwrap(), vec![ - "year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet".to_string(), - "year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet".to_string(), - "year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet".to_string(), - "year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet".to_string() + Path::from("year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet"), + Path::from("year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet") ] ); @@ -298,10 +300,10 @@ async fn read_delta_8_0_table_with_partitions() { assert_eq!( table.get_files_by_partitions(&filters).unwrap(), vec![ - "year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet".to_string(), - "year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet".to_string(), - "year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet".to_string(), - "year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet".to_string() + Path::from("year=2020/month=2/day=3/part-00000-94d16827-f2fd-42cd-a060-f67ccc63ced9.c000.snappy.parquet"), + Path::from("year=2020/month=2/day=5/part-00000-89cdd4c8-2af7-4add-8ea3-3990b2f027b5.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=20/part-00000-9275fdf4-3961-4184-baa0-1c8a2bb98104.c000.snappy.parquet"), + Path::from("year=2021/month=12/day=4/part-00000-6dc763c0-3e8b-4d52-b19e-1f92af3fbb25.c000.snappy.parquet") ] ); @@ -312,8 +314,8 @@ async fn read_delta_8_0_table_with_partitions() { assert_eq!( table.get_files_by_partitions(&filters).unwrap(), vec![ - "year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet".to_string(), - "year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet".to_string() + Path::from("year=2020/month=1/day=1/part-00000-8eafa330-3be9-4a39-ad78-fd13c2027c7e.c000.snappy.parquet"), + Path::from("year=2021/month=4/day=5/part-00000-c5856301-3439-4032-a6fc-22b7bc92bebb.c000.snappy.parquet") ] ); } @@ -330,7 +332,9 @@ async fn read_delta_8_0_table_with_null_partition() { }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), - vec!["k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet".to_string()] + vec![Path::from( + "k=A/part-00000-b1f1dbbb-70bc-4970-893f-9bb772bf246e.c000.snappy.parquet" + )] ); let filters = vec![deltalake::PartitionFilter { @@ -340,7 +344,7 @@ async fn read_delta_8_0_table_with_null_partition() { assert_eq!( table.get_files_by_partitions(&filters).unwrap(), vec![ - "k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet".to_string() + Path::from("k=__HIVE_DEFAULT_PARTITION__/part-00001-8474ac85-360b-4f58-b3ea-23990c71b932.c000.snappy.parquet") ] ); } @@ -354,10 +358,12 @@ async fn read_delta_8_0_table_with_special_partition() { assert_eq!( table.get_files(), vec![ - "x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" - .to_string(), - "x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet" - .to_string() + Path::from( + "x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" + ), + Path::from( + "x=B%20B/part-00015-e9abbc6f-85e9-457b-be8e-e9f5b8a22890.c000.snappy.parquet" + ) ] ); @@ -367,10 +373,9 @@ async fn read_delta_8_0_table_with_special_partition() { }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), - vec![ + vec![Path::from( "x=A%2FA/part-00007-b350e235-2832-45df-9918-6cab4f7578f7.c000.snappy.parquet" - .to_string() - ] + )] ); } @@ -386,10 +391,9 @@ async fn read_delta_8_0_table_partition_with_compare_op() { }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), - vec![ + vec![Path::from( "x=9/y=9.9/part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet" - .to_string() - ] + )] ); let filters = vec![deltalake::PartitionFilter { @@ -398,10 +402,9 @@ async fn read_delta_8_0_table_partition_with_compare_op() { }]; assert_eq!( table.get_files_by_partitions(&filters).unwrap(), - vec![ + vec![Path::from( "x=9/y=9.9/part-00007-3c50fba1-4264-446c-9c67-d8e24a1ccf83.c000.snappy.parquet" - .to_string() - ] + )] ); } @@ -447,7 +450,7 @@ async fn test_action_reconciliation() { // Add a file. let a = fs_common::add(3 * 60 * 1000); assert_eq!(1, fs_common::commit_add(&mut table, &a).await); - assert_eq!(table.get_files(), vec![a.path.as_str()]); + assert_eq!(table.get_files(), vec![Path::from(a.path.clone())]); // Remove added file. let r = deltalake::action::Remove { @@ -474,7 +477,7 @@ async fn test_action_reconciliation() { // Add removed file back. assert_eq!(3, fs_common::commit_add(&mut table, &a).await); - assert_eq!(table.get_files(), vec![a.path.as_str()]); + assert_eq!(table.get_files(), vec![Path::from(a.path)]); // tombstone is removed. assert_eq!(table.get_state().all_tombstones().len(), 0); } @@ -509,9 +512,8 @@ async fn test_poll_table_commits() { let is_new = if let PeekCommit::New(version, actions) = peek { assert_eq!(table.version(), 9); - assert!(!table - .get_files_iter() - .any(|f| f == "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet")); + assert!(!table.get_files_iter().any(|f| f + == Path::from("part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet"))); assert_eq!(version, 10); assert_eq!(actions.len(), 2); @@ -519,9 +521,8 @@ async fn test_poll_table_commits() { table.apply_actions(version, actions).unwrap(); assert_eq!(table.version(), 10); - assert!(table - .get_files_iter() - .any(|f| f == "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet")); + assert!(table.get_files_iter().any(|f| f + == Path::from("part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet"))); true } else { diff --git a/rust/tests/read_simple_table_test.rs b/rust/tests/read_simple_table_test.rs index 117a0a934c..06802dc080 100644 --- a/rust/tests/read_simple_table_test.rs +++ b/rust/tests/read_simple_table_test.rs @@ -2,12 +2,15 @@ extern crate chrono; extern crate deltalake; extern crate utime; +use ::object_store::path::Path as ObjectStorePath; use std::path::Path; use self::chrono::{DateTime, FixedOffset, Utc}; #[tokio::test] async fn read_simple_table() { + let current_dir = + ObjectStorePath::from_filesystem_path(std::env::current_dir().unwrap()).unwrap(); let table = deltalake::open_table("./tests/data/simple_table") .await .unwrap(); @@ -19,11 +22,21 @@ async fn read_simple_table() { assert_eq!( files, vec![ - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", + ObjectStorePath::from( + "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet" + ), ] ); let tombstones = table.get_state().all_tombstones(); @@ -35,31 +48,29 @@ async fn read_simple_table() { extended_file_metadata: None, ..Default::default() })); - #[cfg(unix)] - { - let mut paths: Vec = table.get_file_uris().collect(); - paths.sort(); - let expected_paths: Vec = vec![ - "./tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet".to_string(), - "./tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet".to_string(), - "./tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet".to_string(), - "./tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet".to_string(), - "./tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet".to_string() + + let mut paths: Vec = table.get_file_uris().collect(); + paths.sort(); + cfg_if::cfg_if! { + if #[cfg(target_os = "windows")] { + let expected_paths: Vec = vec![ + format!("{}/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", current_dir.as_ref()), + format!("{}/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", current_dir.as_ref()), + format!("{}/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", current_dir.as_ref()), + format!("{}/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", current_dir.as_ref()), + format!("{}/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", current_dir.as_ref()) ]; - assert_eq!(paths, expected_paths); - } - #[cfg(windows)] - { - let mut paths: Vec = table.get_file_uris().collect(); - paths.sort(); - let expected_paths: Vec = vec![ - "./tests/data/simple_table\\part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet".to_string(), - "./tests/data/simple_table\\part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet".to_string(), - "./tests/data/simple_table\\part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet".to_string(), - "./tests/data/simple_table\\part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet".to_string(), - "./tests/data/simple_table\\part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet".to_string() + assert_eq!(paths, expected_paths); + } else { + let expected_paths: Vec = vec![ + format!("/{}/tests/data/simple_table/part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", current_dir.as_ref()), + format!("/{}/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", current_dir.as_ref()), + format!("/{}/tests/data/simple_table/part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", current_dir.as_ref()), + format!("/{}/tests/data/simple_table/part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", current_dir.as_ref()), + format!("/{}/tests/data/simple_table/part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", current_dir.as_ref()) ]; - assert_eq!(paths, expected_paths); + assert_eq!(paths, expected_paths); + } } } @@ -76,12 +87,24 @@ async fn read_simple_table_with_version() { assert_eq!( files, vec![ - "part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet", - "part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet", - "part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet", - "part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet", - "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet", - "part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet", + ObjectStorePath::from( + "part-00000-a72b1fb3-f2df-41fe-a8f0-e65b746382dd-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00001-c506e79a-0bf8-4e2b-a42b-9731b2e490ae-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00003-508ae4aa-801c-4c2c-a923-f6f89930a5c1-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00004-80938522-09c0-420c-861f-5a649e3d9674-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00006-63ce9deb-bc0f-482d-b9a1-7e717b67f294-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00007-94f725e2-3963-4b00-9e83-e31021a93cf9-c000.snappy.parquet" + ), ], ); @@ -96,12 +119,24 @@ async fn read_simple_table_with_version() { assert_eq!( files, vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", + ObjectStorePath::from( + "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00003-53f42606-6cda-4f13-8d07-599a21197296-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet" + ), ] ); @@ -116,12 +151,24 @@ async fn read_simple_table_with_version() { assert_eq!( files, vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", + ObjectStorePath::from( + "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet" + ), + ObjectStorePath::from( + "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet" + ), ], ); } diff --git a/rust/tests/s3_test.rs b/rust/tests/s3_test.rs index 67956602cb..2f875b9b6c 100644 --- a/rust/tests/s3_test.rs +++ b/rust/tests/s3_test.rs @@ -9,6 +9,7 @@ mod s3 { use deltalake::storage; use dynamodb_lock::dynamo_lock_options; use maplit::hashmap; + use object_store::path::Path; use serial_test::serial; /* @@ -46,11 +47,11 @@ mod s3 { assert_eq!( table.get_files(), vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-2befed33-c358-4768-a43c-3eda0d2a499d-c000.snappy.parquet", + 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(); @@ -77,12 +78,12 @@ mod s3 { assert_eq!( table.get_files(), vec![ - "part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet", - "part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet", - "part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet", - "part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet", - "part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet", - "part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet", + Path::from("part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c000.snappy.parquet"), + Path::from("part-00001-7891c33d-cedc-47c3-88a6-abcfb049d3b4-c000.snappy.parquet"), + Path::from("part-00004-315835fe-fb44-4562-98f6-5e6cfa3ae45d-c000.snappy.parquet"), + Path::from("part-00007-3a0e4727-de0d-41b6-81ef-5223cf40f025-c000.snappy.parquet"), + Path::from("part-00000-f17fcbf5-e0dc-40ba-adae-ce66d1fcaef6-c000.snappy.parquet"), + Path::from("part-00001-bb70d2ba-c196-4df2-9c85-f34969ad3aa9-c000.snappy.parquet"), ] ); let tombstones = table.get_state().all_tombstones(); diff --git a/rust/tests/simple_commit_test.rs b/rust/tests/simple_commit_test.rs index 67f63dd530..93434de735 100644 --- a/rust/tests/simple_commit_test.rs +++ b/rust/tests/simple_commit_test.rs @@ -31,18 +31,15 @@ mod simple_commit_s3 { #[tokio::test] #[serial] async fn test_two_commits_s3_fails_with_no_lock() { - use deltalake::StorageError; - let path = "s3://deltars/simple_commit_rw2"; prepare_s3(path).await; std::env::set_var("AWS_S3_LOCKING_PROVIDER", "none "); let result = test_two_commits(path).await; - if let Err(DeltaTableError::StorageError { ref source }) = result { - if let StorageError::S3Generic(err) = source { - assert_eq!(err, "dynamodb locking is not enabled"); - return; - } + if let Err(DeltaTableError::ObjectStore { source: inner }) = result { + let msg = inner.to_string(); + assert!(msg.contains("dynamodb")); + return; } result.unwrap(); diff --git a/rust/tests/vacuum_test.rs b/rust/tests/vacuum_test.rs index 052cc804a3..eaeb77f087 100644 --- a/rust/tests/vacuum_test.rs +++ b/rust/tests/vacuum_test.rs @@ -1,9 +1,7 @@ use chrono::Duration; -use deltalake::storage::file::FileStorageBackend; use deltalake::storage::StorageError; use deltalake::vacuum::Clock; use deltalake::vacuum::Vacuum; -use deltalake::StorageBackend; use std::sync::Arc; use common::clock::TestClock; @@ -15,8 +13,7 @@ mod common; #[tokio::test] async fn vacuum_delta_8_0_table() { - let backend = FileStorageBackend::new(""); - let mut table = deltalake::open_table(&backend.join_paths(&["tests", "data", "delta-0.8.0"])) + let mut table = deltalake::open_table("./tests/data/delta-0.8.0") .await .unwrap(); @@ -44,12 +41,7 @@ async fn vacuum_delta_8_0_table() { // do not enforce retention duration check with 0 hour will purge all files assert_eq!( result.files_deleted, - vec![backend.join_paths(&[ - "tests", - "data", - "delta-0.8.0", - "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet", - ])] + vec!["part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"] ); let result = Vacuum::default() @@ -61,12 +53,7 @@ async fn vacuum_delta_8_0_table() { assert_eq!( result.files_deleted, - vec![backend.join_paths(&[ - "tests", - "data", - "delta-0.8.0", - "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet", - ])] + vec!["part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet"] ); let retention_hours = SystemTime::now() @@ -127,8 +114,6 @@ async fn test_non_partitioned_table() { assert!(!is_deleted(&mut context, "dont_delete_me.parquet").await); } -// TODO: See #682. Issues with deleting since windows uses \ for paths -#[cfg(not(target_os = "windows"))] #[tokio::test] // Validate vacuum works on a table with multiple partitions async fn test_partitioned_table() { @@ -179,8 +164,6 @@ async fn test_partitioned_table() { assert!(!is_deleted(&mut context, "date=2022-07-03/x=2/dont_delete_me.parquet").await); } -// TODO: See #682. Issues with deleting since windows uses \ for paths -#[cfg(not(target_os = "windows"))] #[tokio::test] // Partitions that start with _ are not ignored async fn test_partitions_included() { @@ -309,7 +292,7 @@ async fn test_non_managed_files() { } async fn is_deleted(context: &mut TestContext, path: &str) -> bool { - let uri = context.table.as_ref().unwrap().table_uri.to_string(); + let uri = context.table.as_ref().unwrap().table_uri.clone(); let backend = context.get_storage(); let path = uri + "/" + path; let res = backend.head_obj(&path).await;