From 99cd80407d49279d61ad0e5e511a1d8fd4c28866 Mon Sep 17 00:00:00 2001 From: Xu Chen Date: Wed, 27 Dec 2023 14:34:53 +0800 Subject: [PATCH] [NativeIO] Enable parquet rowgroup prefetch. Support s3 host style access (#384) * enable prefetch. support s3 host style access Signed-off-by: chenxu * rustfmt for all Signed-off-by: chenxu * fix rust clippy error Signed-off-by: chenxu * fix: use s3 path style access for presto ci Signed-off-by: chenxu * fix more clippy issues Signed-off-by: chenxu --------- Signed-off-by: chenxu Co-authored-by: chenxu --- .../apache/flink/lakesoul/tool/FlinkUtil.java | 4 +- .../presto/lakesoul/LakeSoulConfig.java | 9 + .../presto/lakesoul/LakeSoulRecordCursor.java | 3 +- .../spark/sql/vectorized/NativeIOUtils.scala | 11 +- .../lakesoul/lakesoul/io/NativeIOBase.java | 4 +- rust/Cargo.lock | 582 ++-- rust/Cargo.toml | 26 +- rust/lakesoul-datafusion/src/catalog/mod.rs | 136 +- .../datasource/file_format/metadata_format.rs | 190 +- .../lakesoul-datafusion/src/datasource/mod.rs | 2 +- .../src/datasource/table_provider.rs | 48 +- rust/lakesoul-datafusion/src/error.rs | 14 +- .../src/lakesoul_table/helpers.rs | 25 +- .../src/lakesoul_table/mod.rs | 65 +- rust/lakesoul-datafusion/src/lib.rs | 9 +- rust/lakesoul-datafusion/src/planner/mod.rs | 2 +- .../src/planner/physical_planner.rs | 53 +- .../src/planner/query_planner.rs | 13 +- .../src/serialize/arrow_java.rs | 285 +- rust/lakesoul-datafusion/src/serialize/mod.rs | 2 +- .../src/test/insert_tests.rs | 437 +-- rust/lakesoul-datafusion/src/test/mod.rs | 4 +- .../src/test/upsert_tests.rs | 2693 ++++++++++++----- rust/lakesoul-io-c/build.rs | 2 +- .../src/datasource/empty_schema.rs | 5 +- .../lakesoul-io/src/datasource/file_format.rs | 105 +- rust/lakesoul-io/src/datasource/listing.rs | 82 +- rust/lakesoul-io/src/datasource/mod.rs | 4 +- .../src/datasource/parquet_source.rs | 21 +- .../datasource/physical_plan/empty_schema.rs | 10 +- .../src/datasource/physical_plan/merge.rs | 51 +- .../src/datasource/physical_plan/mod.rs | 4 +- rust/lakesoul-io/src/filter/parser.rs | 4 +- rust/lakesoul-io/src/hdfs/mod.rs | 2 +- rust/lakesoul-io/src/hdfs/util.rs | 20 +- rust/lakesoul-io/src/helpers.rs | 43 +- rust/lakesoul-io/src/lakesoul_io_config.rs | 61 +- rust/lakesoul-io/src/lakesoul_reader.rs | 13 +- rust/lakesoul-io/src/lakesoul_writer.rs | 20 +- rust/lakesoul-io/src/lib.rs | 21 +- rust/lakesoul-io/src/projection/mod.rs | 12 +- .../src/repartition/distributor_channels.rs | 21 +- rust/lakesoul-io/src/repartition/mod.rs | 251 +- .../src/sorted_merge/merge_operator.rs | 3 +- rust/lakesoul-io/src/sorted_merge/mod.rs | 4 +- .../src/sorted_merge/sorted_stream_merger.rs | 1 - rust/lakesoul-metadata-c/src/lib.rs | 147 +- rust/lakesoul-metadata/src/error.rs | 27 +- rust/lakesoul-metadata/src/lib.rs | 949 +++--- rust/lakesoul-metadata/src/metadata_client.rs | 345 ++- rust/proto/build.rs | 7 +- rust/proto/src/lib.rs | 4 +- .../presto/catalog/lakesoul.properties | 1 + 53 files changed, 4145 insertions(+), 2712 deletions(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java index 97f6222c1..09927e326 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java @@ -429,10 +429,13 @@ public static void setFSConfigs(Configuration conf, NativeIOBase io) { setFSConf(conf, "fs.s3a.secret.key", "fs.s3a.secret.key", io); setFSConf(conf, "fs.s3a.endpoint", "fs.s3a.endpoint", io); setFSConf(conf, "fs.s3a.endpoint.region", "fs.s3a.endpoint.region", io); + setFSConf(conf, "fs.s3a.path.style.access", "fs.s3a.path.style.access", io); // try flink's s3 credential configs setFSConf(conf, "s3.access-key", "fs.s3a.access.key", io); setFSConf(conf, "s3.secret-key", "fs.s3a.secret.key", io); setFSConf(conf, "s3.endpoint", "fs.s3a.endpoint", io); + setFSConf(conf, "s3.endpoint.region", "fs.s3a.endpoint.region", io); + setFSConf(conf, "s3.path.style.access", "fs.s3a.path.style.access", io); } public static void setFSConf(Configuration conf, String confKey, String fsConfKey, NativeIOBase io) { @@ -443,7 +446,6 @@ public static void setFSConf(Configuration conf, String confKey, String fsConfKe } } - public static Object convertStringToInternalValue(String valStr, LogicalType type) { if (valStr == null) { return null; diff --git a/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulConfig.java b/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulConfig.java index 9b1545a3a..55c3d9cba 100644 --- a/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulConfig.java +++ b/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulConfig.java @@ -29,6 +29,7 @@ private LakeSoulConfig(Map config){ this.endpoint = config.get("fs.s3a.endpoint"); this.defaultFS = config.get("fs.defaultFS"); this.user = config.get("fs.hdfs.user"); + this.virtualPathStyle = Boolean.parseBoolean(config.getOrDefault("fs.s3a.path.style.access", "false")); this.timeZone = config.getOrDefault("timezone",""); } @@ -40,6 +41,7 @@ private LakeSoulConfig(Map config){ private String user; private String defaultFS; private String timeZone; + private boolean virtualPathStyle; public String getAccessKey() { @@ -106,4 +108,11 @@ public void setTimeZone(String timeZone) { this.timeZone = timeZone; } + public boolean isVirtualPathStyle() { + return virtualPathStyle; + } + + public void setVirtualPathStyle(boolean virtualPathStyle) { + this.virtualPathStyle = virtualPathStyle; + } } diff --git a/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulRecordCursor.java b/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulRecordCursor.java index beafb0334..b340a2907 100644 --- a/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulRecordCursor.java +++ b/lakesoul-presto/src/main/java/com/facebook/presto/lakesoul/LakeSoulRecordCursor.java @@ -97,7 +97,8 @@ public LakeSoulRecordCursor(LakeSoulRecordSet recordSet) throws IOException { LakeSoulConfig.getInstance().getBucketName(), LakeSoulConfig.getInstance().getEndpoint(), LakeSoulConfig.getInstance().getDefaultFS(), - LakeSoulConfig.getInstance().getUser() + LakeSoulConfig.getInstance().getUser(), + LakeSoulConfig.getInstance().isVirtualPathStyle() ); // init reader diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala index 1a4571b1d..43728e986 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala @@ -31,7 +31,8 @@ class NativeIOOptions(val s3Bucket: String, val s3Endpoint: String, val s3Region: String, val fsUser: String, - val defaultFS: String + val defaultFS: String, + val virtual_path_style: Boolean ) object NativeIOUtils{ @@ -69,11 +70,12 @@ object NativeIOUtils{ val s3aRegion = taskAttemptContext.getConfiguration.get("fs.s3a.endpoint.region") val s3aAccessKey = taskAttemptContext.getConfiguration.get("fs.s3a.access.key") val s3aSecretKey = taskAttemptContext.getConfiguration.get("fs.s3a.secret.key") - return new NativeIOOptions(awsS3Bucket, s3aAccessKey, s3aSecretKey, s3aEndpoint, s3aRegion, user, defaultFS) + val virtualPathStyle = taskAttemptContext.getConfiguration.getBoolean("fs.s3a.path.style.access", false) + return new NativeIOOptions(awsS3Bucket, s3aAccessKey, s3aSecretKey, s3aEndpoint, s3aRegion, user, defaultFS, virtualPathStyle) case _ => } } - new NativeIOOptions(null, null, null, null, null, user, defaultFS) + new NativeIOOptions(null, null, null, null, null, user, defaultFS, false) } def setNativeIOOptions(nativeIO: NativeIOBase, options: NativeIOOptions): Unit = { @@ -84,7 +86,8 @@ object NativeIOUtils{ options.s3Bucket, options.s3Endpoint, options.fsUser, - options.defaultFS + options.defaultFS, + options.virtual_path_style ) } diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java index b02a76e77..643b80379 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java @@ -105,7 +105,8 @@ public void setBufferSize(int bufferSize) { public void setObjectStoreOptions(String accessKey, String accessSecret, String region, String bucketName, String endpoint, - String user, String defaultFS) { + String user, String defaultFS, + boolean virtual_path_style) { setObjectStoreOption("fs.s3a.access.key", accessKey); setObjectStoreOption("fs.s3a.secret.key", accessSecret); setObjectStoreOption("fs.s3a.endpoint.region", region); @@ -113,6 +114,7 @@ public void setObjectStoreOptions(String accessKey, String accessSecret, setObjectStoreOption("fs.s3a.endpoint", endpoint); setObjectStoreOption("fs.defaultFS", defaultFS); setObjectStoreOption("fs.hdfs.user", user); + setObjectStoreOption("fs.s3a.path.style.access", String.valueOf(virtual_path_style)); } public void setObjectStoreOption(String key, String value) { diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 4973d4045..cf9600f92 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -78,9 +78,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.75" +version = "1.0.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" +checksum = "59d2a3357dde987206219e78ecfbbb6e8dad06cbb65292758d3270e6254f7355" [[package]] name = "array-init" @@ -103,8 +103,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8919668503a4f2d8b6da96fa7c16e93046bfb3412ffcfa1e5dc7d2e3adcb378" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "ahash", "arrow-arith", @@ -125,8 +124,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef983914f477d4278b068f13b3224b7d19eb2b807ac9048544d3bfebdf2554c4" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -140,8 +138,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6eaf89041fa5937940ae390294ece29e1db584f46d995608d6e5fe65a2e0e9b" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "ahash", "arrow-buffer", @@ -150,7 +147,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "num", "packed_simd", ] @@ -158,8 +155,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55512d988c6fbd76e514fd3ff537ac50b0a675da5a245e4fdad77ecfd654205f" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "bytes", "half", @@ -169,8 +165,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "655ee51a2156ba5375931ce21c1b2494b1d9260e6dcdc6d4db9060c37dc3325b" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -187,8 +182,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "258bb689997ad5b6660b3ce3638bd6b383d668ec555ed41ad7c6559cbb2e4f91" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -206,8 +200,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dc2b9fec74763427e2e5575b8cc31ce96ba4c9b4eb05ce40e0616d9fad12461" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-buffer", "arrow-schema", @@ -218,8 +211,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eaa6ab203cc6d89b7eaa1ac781c1dfeef325454c5d5a0419017f95e6bafc03c" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -232,8 +224,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb64e30d9b73f66fdc5c52d5f4cf69bbf03d62f64ffeafa0715590a5320baed7" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -252,8 +243,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9a818951c0d11c428dda03e908175969c262629dd20bd0850bd6c7a8c3bfe48" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -267,8 +257,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5d664318bc05f930559fc088888f0f7174d3c5bc888c0f4f9ae8f23aa398ba3" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "ahash", "arrow-array", @@ -276,14 +265,13 @@ dependencies = [ "arrow-data", "arrow-schema", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", ] [[package]] name = "arrow-schema" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aaf4d737bba93da59f16129bec21e087aed0be84ff840e74146d4703879436cb" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "bitflags 2.4.1", "serde", @@ -292,8 +280,7 @@ dependencies = [ [[package]] name = "arrow-select" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "374c4c3b812ecc2118727b892252a4a4308f87a8aca1dbf09f3ce4bc578e668a" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "ahash", "arrow-array", @@ -306,8 +293,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15aed5624bb23da09142f58502b59c23f5bea607393298bb81dab1ce60fc769" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "arrow-array", "arrow-buffer", @@ -321,20 +307,22 @@ dependencies = [ [[package]] name = "async-channel" -version = "1.9.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81953c529336010edd6d8e358f886d9581267795c61b19475b71314bffa46d35" +checksum = "1ca33f4bc4ed1babef42cad36cc1f51fa88be00420404e5b1e80ab1b18f7678c" dependencies = [ "concurrent-queue", - "event-listener", + "event-listener 4.0.1", + "event-listener-strategy", "futures-core", + "pin-project-lite", ] [[package]] name = "async-compression" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" +checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" dependencies = [ "bzip2", "flate2", @@ -354,24 +342,35 @@ version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "287272293e9d8c41773cec55e365490fe034813a2f172f502d6ddcf75b2f582b" dependencies = [ - "event-listener", + "event-listener 2.5.3", +] + +[[package]] +name = "async-lock" +version = "3.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7125e42787d53db9dd54261812ef17e937c95a51e4d291373b670342fa44310c" +dependencies = [ + "event-listener 4.0.1", + "event-listener-strategy", + "pin-project-lite", ] [[package]] name = "async-task" -version = "4.5.0" +version = "4.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4eb2cdb97421e01129ccb49169d8279ed21e829929144f4a22a6e54ac549ca1" +checksum = "e1d90cd0b264dfdd8eb5bad0a2c217c1f88fa96a8573f40e7b12de23fb468f46" [[package]] name = "async-trait" -version = "0.1.74" +version = "0.1.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" +checksum = "fdf6721fb0140e4f897002dd086c06f6c27775df19cfe1fccb21181a48fd2c98" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -478,12 +477,12 @@ dependencies = [ [[package]] name = "blocking" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c36a4d0d48574b3dd360b4b7d95cc651d2b6557b6402848a27d4b228a473e2a" +checksum = "6a37913e8dc4ddcc604f0c6d3bf2887c995153af3611de9e23c352b44c1b9118" dependencies = [ "async-channel", - "async-lock", + "async-lock 3.2.0", "async-task", "fastrand", "futures-io", @@ -600,7 +599,7 @@ dependencies = [ "pure-rust-locales", "serde", "wasm-bindgen", - "windows-targets", + "windows-targets 0.48.5", ] [[package]] @@ -674,9 +673,9 @@ dependencies = [ [[package]] name = "concurrent-queue" -version = "2.3.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f057a694a54f12365049b0958a1685bb52d567f5593b355fbf685838e873d400" +checksum = "d16048cd947b08fa32c24458a22f5dc5e835264f689f4f5653210c69fd107363" dependencies = [ "crossbeam-utils", ] @@ -709,9 +708,9 @@ checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" [[package]] name = "core-foundation" -version = "0.9.3" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "194a7a9e6de53fa55116934067c844d9d749312f75c6f6d0980e8c252f8c2146" +checksum = "91e195e091a93c46f7102ec7818a2aa394e1e1771c3ab4825963fa03e45afb8f" dependencies = [ "core-foundation-sys", "libc", @@ -719,9 +718,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.4" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" +checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" [[package]] name = "cpufeatures" @@ -743,9 +742,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.16" +version = "0.8.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a22b2d63d4d1dc0b7f1b6b2747dd0088008a9be28b6ddf0b1e7d335e3037294" +checksum = "c3a430a770ebd84726f584a90ee7f020d28db52c6d02138900f22341f866d39c" dependencies = [ "cfg-if", ] @@ -814,12 +813,12 @@ dependencies = [ [[package]] name = "ctor" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37e366bff8cd32dd8754b0991fb66b279dc48f598c3a18914852a6673deef583" +checksum = "30d2b3721e861707777e3195b0158f950ae6dc4a27e4d02ff9f67e3eb3de199e" dependencies = [ "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -835,7 +834,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "lock_api", "once_cell", "parking_lot_core", @@ -844,8 +843,7 @@ dependencies = [ [[package]] name = "datafusion" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "676796427e638d85e9eadf13765705212be60b34f8fc5d3934d95184c63ca1b4" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "ahash", "arrow", @@ -868,7 +866,7 @@ dependencies = [ "futures", "glob", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "indexmap 2.1.0", "itertools 0.11.0", "log", @@ -891,8 +889,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e23b3d21a6531259d291bd20ce59282ea794bda1018b0a1e278c13cd52e50c" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "ahash", "arrow", @@ -910,8 +907,7 @@ dependencies = [ [[package]] name = "datafusion-execution" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4de1fd0d8db0f2b8e4f4121bfa1c7c09d3a5c08a0a65c2229cd849eb65cff855" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "arrow", "chrono", @@ -919,7 +915,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "log", "object_store", "parking_lot", @@ -931,8 +927,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18e227fe88bf6730cab378d0cd8fc4c6b2ea42bc7e414a8ea9feba7225932735" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "ahash", "arrow", @@ -946,8 +941,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c6648e62ea7605b9bfcd87fdc9d67e579c3b9ac563a87734ae5fe6d79ee4547" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "arrow", "async-trait", @@ -955,7 +949,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "itertools 0.11.0", "log", "regex-syntax", @@ -964,8 +958,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f32b8574add16a32411a9b3fb3844ac1fc09ab4e7be289f86fd56d620e4f2508" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "ahash", "arrow", @@ -980,7 +973,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "hex", "indexmap 2.1.0", "itertools 0.11.0", @@ -999,8 +992,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "796abd77d5bfecd9e5275a99daf0ec45f5b3a793ec431349ce8211a67826fd22" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "ahash", "arrow", @@ -1015,7 +1007,7 @@ dependencies = [ "datafusion-physical-expr", "futures", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "indexmap 2.1.0", "itertools 0.11.0", "log", @@ -1030,8 +1022,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "33.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced70b8a5648ba7b95c61fc512183c33287ffe2c9f22ffe22700619d7d48c84f" +source = "git+https://github.com/lakesoul-io/arrow-datafusion.git?branch=datafusion-33-parquet-prefetch#235eb27b6b0d23b18fb4a111fecbf5fa1b0d46a2" dependencies = [ "arrow", "arrow-schema", @@ -1092,12 +1083,12 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.5" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3e13f66a2f95e32a39eaa81f6b95d42878ca0e1db0c7543723dfe12557e860" +checksum = "a258e46cdc063eb8519c00b9fc845fc47bcfca4130e2f08e88665ceda8474245" dependencies = [ "libc", - "windows-sys", + "windows-sys 0.52.0", ] [[package]] @@ -1106,6 +1097,27 @@ version = "2.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" +[[package]] +name = "event-listener" +version = "4.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "84f2cdcf274580f2d63697192d744727b3198894b1bf02923643bf59e2c26712" +dependencies = [ + "concurrent-queue", + "parking", + "pin-project-lite", +] + +[[package]] +name = "event-listener-strategy" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "958e4d70b6d5e81971bebec42271ec641e7ff4e170a6fa605f2b8a8b65cb97d3" +dependencies = [ + "event-listener 4.0.1", + "pin-project-lite", +] + [[package]] name = "fallible-iterator" version = "0.2.0" @@ -1158,18 +1170,18 @@ checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" [[package]] name = "form_urlencoded" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a62bc1cf6f830c2ec14a513a9fb124d0a213a629668a4186f329db21fe045652" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" dependencies = [ "percent-encoding", ] [[package]] name = "futures" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" dependencies = [ "futures-channel", "futures-core", @@ -1182,9 +1194,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" dependencies = [ "futures-core", "futures-sink", @@ -1192,15 +1204,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" [[package]] name = "futures-executor" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" dependencies = [ "futures-core", "futures-task", @@ -1209,15 +1221,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" [[package]] name = "futures-lite" -version = "1.13.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49a9d51ce47660b1e808d3c990b4709f2f415d928835a17dfd16991515c46bce" +checksum = "aeee267a1883f7ebef3700f262d2d54de95dfaf38189015a74fdc4e0c7ad8143" dependencies = [ "futures-core", "pin-project-lite", @@ -1225,32 +1237,32 @@ dependencies = [ [[package]] name = "futures-macro" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] name = "futures-sink" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" [[package]] name = "futures-task" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" [[package]] name = "futures-util" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" dependencies = [ "futures-channel", "futures-core", @@ -1276,9 +1288,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +checksum = "fe9006bed769170c11f845cf00c7c1e9092aeb3f268e007c3e760ac68008070f" dependencies = [ "cfg-if", "libc", @@ -1287,9 +1299,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.28.0" +version = "0.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" +checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" [[package]] name = "glob" @@ -1299,9 +1311,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.21" +version = "0.3.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" +checksum = "4d6250322ef6e60f93f9a2162799302cd6f68f79f6e5d85c8c16f14d1d958178" dependencies = [ "bytes", "fnv", @@ -1309,7 +1321,7 @@ dependencies = [ "futures-sink", "futures-util", "http", - "indexmap 1.9.3", + "indexmap 2.1.0", "slab", "tokio", "tokio-util", @@ -1335,9 +1347,9 @@ checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" [[package]] name = "hashbrown" -version = "0.14.2" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" +checksum = "290f1a1d9242c78d09ce40a5e87e7554ee637af1351968159f4952f028f75604" dependencies = [ "ahash", "allocator-api2", @@ -1356,7 +1368,7 @@ name = "hdrs" version = "0.2.0" source = "git+https://github.com/lakesoul-io/hdrs.git?branch=main#acebdca01cd33aa2fadbb970b87ec9aaa65b841f" dependencies = [ - "async-lock", + "async-lock 2.8.0", "blocking", "errno", "futures", @@ -1403,18 +1415,18 @@ dependencies = [ [[package]] name = "home" -version = "0.5.5" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5444c27eef6923071f7ebcc33e3444508466a76f7a2b93da00ed6e19f30c1ddb" +checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" dependencies = [ - "windows-sys", + "windows-sys 0.52.0", ] [[package]] name = "http" -version = "0.2.9" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" +checksum = "8947b1a6fad4393052c7ba1f4cd97bed3e953a95c79c92ad9b051a04611d9fbb" dependencies = [ "bytes", "fnv", @@ -1423,9 +1435,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" +checksum = "7ceab25649e9960c0311ea418d17bee82c0dcec1bd053b5f9a66e265a693bed2" dependencies = [ "bytes", "http", @@ -1452,9 +1464,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.27" +version = "0.14.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffb1cfd654a8219eaef89881fdb3bb3b1cdc5fa75ded05d6933b2b382e395468" +checksum = "bf96e135eb83a2a8ddf766e426a841d8ddd7449d5f00d34ea02b41d2f19eef80" dependencies = [ "bytes", "futures-channel", @@ -1467,7 +1479,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.10", + "socket2", "tokio", "tower-service", "tracing", @@ -1513,9 +1525,9 @@ dependencies = [ [[package]] name = "idna" -version = "0.4.0" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d20d6b07bfbc108882d88ed8e37d39636dcc260e15e30c45e6ba089610b917c" +checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" dependencies = [ "unicode-bidi", "unicode-normalization", @@ -1538,7 +1550,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d530e1a18b1cb4c484e6e34556a0d948706958449fca0cab753d649f2bce3d1f" dependencies = [ "equivalent", - "hashbrown 0.14.2", + "hashbrown 0.14.3", ] [[package]] @@ -1573,9 +1585,9 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" +checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" [[package]] name = "jobserver" @@ -1588,9 +1600,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.64" +version = "0.3.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" +checksum = "cee9c64da59eae3b50095c18d3e74f8b73c0b86d2792824ff01bbce68ba229ca" dependencies = [ "wasm-bindgen", ] @@ -1774,9 +1786,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.149" +version = "0.2.151" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a08173bc88b7955d1b3145aa561539096c421ac8debde8cbc3612ec635fee29b" +checksum = "302d7ab3130588088d277783b1e2d2e10c9e9e4a16dd9050e6ec93fb3e7048f4" [[package]] name = "libm" @@ -1786,9 +1798,9 @@ checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" [[package]] name = "linux-raw-sys" -version = "0.4.10" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" +checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" [[package]] name = "lock_api" @@ -1859,14 +1871,14 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.9" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" +checksum = "8f3d0b296e374a4e6f3c7b0a1f5a51d748a0d34c85e7dc48fc3fa9a87657fe09" dependencies = [ "libc", "log", "wasi", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -1985,9 +1997,9 @@ dependencies = [ [[package]] name = "object" -version = "0.32.1" +version = "0.32.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" dependencies = [ "memchr", ] @@ -1995,8 +2007,7 @@ dependencies = [ [[package]] name = "object_store" version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f930c88a43b1c3f6e776dfe495b4afab89882dbc81530c632db2ed65451ebcb4" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=object_store_0.7_opt#3fd4b6ba312ddc18b7f8ce509350546ef03c7ae6" dependencies = [ "async-trait", "base64", @@ -2023,9 +2034,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "ordered-float" @@ -2052,6 +2063,12 @@ dependencies = [ "num-traits", ] +[[package]] +name = "parking" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" + [[package]] name = "parking_lot" version = "0.12.1" @@ -2072,14 +2089,13 @@ dependencies = [ "libc", "redox_syscall", "smallvec", - "windows-targets", + "windows-targets 0.48.5", ] [[package]] name = "parquet" version = "48.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bfe55df96e3f02f11bf197ae37d91bb79801631f82f6195dd196ef521df3597" +source = "git+https://github.com/lakesoul-io/arrow-rs.git?branch=arrow-rs-48-parquet-bufferred#f00cb1b172856e37b0a1eca80832caceadc276b3" dependencies = [ "ahash", "arrow-array", @@ -2095,7 +2111,7 @@ dependencies = [ "chrono", "flate2", "futures", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "lz4_flex", "num", "num-bigint", @@ -2126,9 +2142,9 @@ checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" [[package]] name = "percent-encoding" -version = "2.3.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "petgraph" @@ -2203,9 +2219,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" +checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" [[package]] name = "postgres" @@ -2230,7 +2246,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -2295,9 +2311,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.69" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" +checksum = "75cb1540fadbd5b8fbccc4dddad2734eba435053f725621c070711a14bb5f4b8" dependencies = [ "unicode-ident", ] @@ -2470,9 +2486,9 @@ checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" [[package]] name = "reqwest" -version = "0.11.22" +version = "0.11.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" +checksum = "37b1ae8d9ac08420c66222fb9096fc5de435c3c48542bc5336c51892cffafb41" dependencies = [ "base64", "bytes", @@ -2527,16 +2543,16 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.5" +version = "0.17.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" +checksum = "688c63d65483050968b2a8937f7995f443e27041a0f7700aa59b0822aedebb74" dependencies = [ "cc", "getrandom", "libc", "spin 0.9.8", "untrusted 0.9.0", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -2556,34 +2572,34 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.21" +version = "0.38.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" +checksum = "72e572a5e8ca657d7366229cdde4bd14c4eb5499a9573d4d366fe1b599daa316" dependencies = [ "bitflags 2.4.1", "errno", "libc", "linux-raw-sys", - "windows-sys", + "windows-sys 0.52.0", ] [[package]] name = "rustls" -version = "0.21.8" +version = "0.21.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" +checksum = "f9d5a6813c0759e4609cd494e8e725babae6a2ca7b62a5536a13daaec6fcb7ba" dependencies = [ "log", - "ring 0.17.5", + "ring 0.17.7", "rustls-webpki", "sct", ] [[package]] name = "rustls-pemfile" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d3987094b1d07b653b7dfdc3f70ce9a1da9c51ac18c1b06b662e4f9a0e9f4b2" +checksum = "1c74cae0a4cf6ccbbf5f359f08efdf8ee7e1dc532573bf0db71968cb56b1448c" dependencies = [ "base64", ] @@ -2594,7 +2610,7 @@ version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring 0.17.5", + "ring 0.17.7", "untrusted 0.9.0", ] @@ -2606,9 +2622,9 @@ checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" [[package]] name = "ryu" -version = "1.0.15" +version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" +checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" [[package]] name = "same-file" @@ -2631,7 +2647,7 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.17.5", + "ring 0.17.7", "untrusted 0.9.0", ] @@ -2649,22 +2665,22 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.190" +version = "1.0.193" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91d3c334ca1ee894a2c6f6ad698fe8c435b76d504b13d436f0685d648d6d96f7" +checksum = "25dd9975e68d0cb5aa1120c288333fc98731bd1dd12f561e468ea4728c042b89" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.190" +version = "1.0.193" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67c5609f394e5c2bd7fc51efda478004ea80ef42fee983d5c67a65e34f32c0e3" +checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -2748,9 +2764,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.11.1" +version = "1.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "942b4a808e05215192e39f4ab80813e599068285906cc91aa64f923db842bd5a" +checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" [[package]] name = "snafu" @@ -2776,19 +2792,9 @@ dependencies = [ [[package]] name = "snap" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" - -[[package]] -name = "socket2" -version = "0.4.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f7916fc008ca5542385b89a3d3ce689953c143e9304a9bf8beec1de48994c0d" -dependencies = [ - "libc", - "winapi", -] +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "socket2" @@ -2797,7 +2803,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" dependencies = [ "libc", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -2894,7 +2900,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -2916,9 +2922,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.38" +version = "2.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e96b79aaa137db8f61e26363a0c9b47d8b4ec75da28b7d1d614c2303e232408b" +checksum = "ee659fb5f3d355364e1f3e5bc10fb82068efbf824a1e9d1c9504244a6469ad53" dependencies = [ "proc-macro2", "quote", @@ -2956,14 +2962,14 @@ dependencies = [ "fastrand", "redox_syscall", "rustix", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] name = "termcolor" -version = "1.3.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6093bad37da69aab9d123a8091e4be0aa4a03e4d601ec641c327398315f62b64" +checksum = "ff1bc3d3f05aff0403e8ac0d92ced918ec05b666a43f83297ccef5bea8a3d449" dependencies = [ "winapi-util", ] @@ -3011,9 +3017,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.33.0" +version = "1.35.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f38200e3ef7995e5ef13baec2f432a6da0aa9ac495b2c0e8f3b7eec2c92d653" +checksum = "c89b4efa943be685f629b149f53829423f8f5531ea21249408e8e2f8671ec104" dependencies = [ "backtrace", "bytes", @@ -3023,20 +3029,20 @@ dependencies = [ "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.5.5", + "socket2", "tokio-macros", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] name = "tokio-macros" -version = "2.1.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" +checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -3059,7 +3065,7 @@ dependencies = [ "postgres-protocol", "postgres-types", "rand", - "socket2 0.5.5", + "socket2", "tokio", "tokio-util", "whoami", @@ -3152,7 +3158,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -3166,9 +3172,9 @@ dependencies = [ [[package]] name = "try-lock" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" +checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "twox-hash" @@ -3188,9 +3194,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" +checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" [[package]] name = "unicode-ident" @@ -3233,9 +3239,9 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.4.1" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "143b538f18257fac9cad154828a57c6bf5157e1aa604d4816b5995bf6de87ae5" +checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" dependencies = [ "form_urlencoded", "idna", @@ -3244,9 +3250,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.5.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" +checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" dependencies = [ "getrandom", "rand", @@ -3255,13 +3261,13 @@ dependencies = [ [[package]] name = "uuid-macro-internal" -version = "1.5.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d8c6bba9b149ee82950daefc9623b32bb1dacbfb1890e352f6b887bd582adaf" +checksum = "f49e7f3f3db8040a100710a11932239fd30697115e2ba4107080d8252939845e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] @@ -3297,9 +3303,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.87" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" +checksum = "0ed0d4f68a3015cc185aff4db9506a015f4b96f95303897bfa23f846db54064e" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3307,24 +3313,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.87" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" +checksum = "1b56f625e64f3a1084ded111c4d5f477df9f8c92df113852fa5a374dbda78826" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.37" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c02dbc21516f9f1f04f187958890d7e6026df8d16540b7ad9492bc34a67cea03" +checksum = "ac36a15a220124ac510204aec1c3e5db8a22ab06fd6706d881dc6149f8ed9a12" dependencies = [ "cfg-if", "js-sys", @@ -3334,9 +3340,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.87" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" +checksum = "0162dbf37223cd2afce98f3d0785506dcb8d266223983e4b5b525859e6e182b2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3344,22 +3350,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.87" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" +checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.87" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" +checksum = "7ab9b36309365056cd639da3134bf87fa8f3d86008abf99e612384a6eecd459f" [[package]] name = "wasm-streams" @@ -3376,9 +3382,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.64" +version = "0.3.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b85cbef8c220a6abc02aefd892dfc0fc23afb1c6a426316ec33253a3877249b" +checksum = "50c24a44ec86bb68fbecd1b3efed7e85ea5621b39b35ef2766b66cd984f8010f" dependencies = [ "js-sys", "wasm-bindgen", @@ -3386,9 +3392,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.25.2" +version = "0.25.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14247bb57be4f377dfb94c72830b8ce8fc6beac03cf4bf7b9732eadd414123fc" +checksum = "1778a42e8b3b90bff8d0f5032bf22250792889a5cdc752aa0020c84abe3aaf10" [[package]] name = "which" @@ -3449,7 +3455,7 @@ version = "0.51.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" dependencies = [ - "windows-targets", + "windows-targets 0.48.5", ] [[package]] @@ -3458,7 +3464,16 @@ version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" dependencies = [ - "windows-targets", + "windows-targets 0.48.5", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.0", ] [[package]] @@ -3467,13 +3482,28 @@ version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" dependencies = [ - "windows_aarch64_gnullvm", - "windows_aarch64_msvc", - "windows_i686_gnu", - "windows_i686_msvc", - "windows_x86_64_gnu", - "windows_x86_64_gnullvm", - "windows_x86_64_msvc", + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", +] + +[[package]] +name = "windows-targets" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" +dependencies = [ + "windows_aarch64_gnullvm 0.52.0", + "windows_aarch64_msvc 0.52.0", + "windows_i686_gnu 0.52.0", + "windows_i686_msvc 0.52.0", + "windows_x86_64_gnu 0.52.0", + "windows_x86_64_gnullvm 0.52.0", + "windows_x86_64_msvc 0.52.0", ] [[package]] @@ -3482,47 +3512,89 @@ version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" + [[package]] name = "windows_aarch64_msvc" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" + [[package]] name = "windows_i686_gnu" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" +[[package]] +name = "windows_i686_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" + [[package]] name = "windows_i686_msvc" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" +[[package]] +name = "windows_i686_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" + [[package]] name = "windows_x86_64_gnu" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" + [[package]] name = "windows_x86_64_gnullvm" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" + [[package]] name = "windows_x86_64_msvc" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" + [[package]] name = "winnow" -version = "0.5.18" +version = "0.5.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "176b6138793677221d420fd2f0aeeced263f197688b36484660da767bca2fa32" +checksum = "9b5c3db89721d50d0e2a673f5043fc4722f76dcc352d7b1ab8b8288bed4ed2c5" dependencies = [ "memchr", ] @@ -3534,7 +3606,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "524e57b2c537c0f9b1e69f1965311ec12182b4122e45035b1508cd24d2adadb1" dependencies = [ "cfg-if", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -3548,22 +3620,22 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.21" +version = "0.7.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "686b7e407015242119c33dab17b8f61ba6843534de936d94368856528eae4dcc" +checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.21" +version = "0.7.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "020f3dfe25dfc38dfea49ce62d5d45ecdd7f0d8a724fa63eb36b6eba4ec76806" +checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.43", ] [[package]] diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 8fbff15c7..7f414451c 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -16,24 +16,14 @@ resolver = "2" [profile.release] [workspace.dependencies] -# datafusion = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-27-parquet-prefetch"} -# datafusion-common = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-27-parquet-prefetch" } -datafusion = "33.0.0" -datafusion-common = "33.0.0" - -# arrow = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-42-parquet-bufferred" } -# arrow-schema = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-42-parquet-bufferred" } -# arrow-array = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-42-parquet-bufferred"} -# arrow-buffer = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-42-parquet-bufferred" } -# parquet = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-42-parquet-bufferred"} -arrow = "48.0.1" -arrow-schema = "48.0.1" -arrow-array = "48.0.1" -arrow-buffer = "48.0.1" -parquet = "48.0.1" - -# object_store = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-42-parquet-bufferred", features = ["aws"] } -object_store = { version = "^0.7.1", features = ["aws"] } +datafusion = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-33-parquet-prefetch"} +datafusion-common = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-33-parquet-prefetch" } +arrow = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } +arrow-schema = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } +arrow-array = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred"} +arrow-buffer = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } +parquet = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred"} +object_store = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "object_store_0.7_opt", features = ["aws", "http"] } tokio-stream = "0.1.9" tokio = { version = "1", features = ["full"] } diff --git a/rust/lakesoul-datafusion/src/catalog/mod.rs b/rust/lakesoul-datafusion/src/catalog/mod.rs index 08de4910e..b92c7d8f2 100644 --- a/rust/lakesoul-datafusion/src/catalog/mod.rs +++ b/rust/lakesoul-datafusion/src/catalog/mod.rs @@ -9,10 +9,10 @@ use std::{env, path::PathBuf}; use lakesoul_io::lakesoul_io_config::{LakeSoulIOConfig, LakeSoulIOConfigBuilder}; use lakesoul_metadata::MetaDataClientRef; -use proto::proto::entity::{TableInfo, DataCommitInfo, DataFileOp, FileOp, CommitOp, Uuid}; +use proto::proto::entity::{CommitOp, DataCommitInfo, DataFileOp, FileOp, TableInfo, Uuid}; use crate::lakesoul_table::helpers::create_io_config_builder_from_table_info; -use crate::serialize::arrow_java::{ArrowJavaSchema, schema_from_metadata_str}; +use crate::serialize::arrow_java::{schema_from_metadata_str, ArrowJavaSchema}; // use crate::transaction::TransactionMetaInfo; use crate::error::Result; @@ -25,33 +25,47 @@ pub struct LakeSoulTableProperty { } pub(crate) async fn create_table(client: MetaDataClientRef, table_name: &str, config: LakeSoulIOConfig) -> Result<()> { - client.create_table( - TableInfo { + client + .create_table(TableInfo { table_id: format!("table_{}", uuid::Uuid::new_v4()), - table_name: table_name.to_string(), + table_name: table_name.to_string(), table_path: format!("file://{}default/{}", env::temp_dir().to_str().unwrap(), table_name), table_schema: serde_json::to_string::(&config.schema().into()).unwrap(), table_namespace: "default".to_string(), - properties: serde_json::to_string(&LakeSoulTableProperty {hash_bucket_num: Some(4)})?, - partitions: format!("{};{}", "", config.primary_keys_slice().iter().map(String::as_str).collect::>().join(",")), + properties: serde_json::to_string(&LakeSoulTableProperty { + hash_bucket_num: Some(4), + })?, + partitions: format!( + "{};{}", + "", + config + .primary_keys_slice() + .iter() + .map(String::as_str) + .collect::>() + .join(",") + ), domain: "public".to_string(), - }).await?; - Ok(()) + }) + .await?; + Ok(()) } -pub(crate) async fn create_io_config_builder(client: MetaDataClientRef, table_name: Option<&str>, fetch_files: bool) -> Result { +pub(crate) async fn create_io_config_builder( + client: MetaDataClientRef, + table_name: Option<&str>, + fetch_files: bool, +) -> Result { if let Some(table_name) = table_name { let table_info = client.get_table_info_by_table_name(table_name, "default").await?; let data_files = if fetch_files { - client.get_data_files_by_table_name(table_name, vec![], "default").await? + client + .get_data_files_by_table_name(table_name, vec![], "default") + .await? } else { vec![] }; - Ok( - create_io_config_builder_from_table_info(Arc::new(table_info)) - .with_files(data_files) - ) - + Ok(create_io_config_builder_from_table_info(Arc::new(table_info)).with_files(data_files)) } else { Ok(LakeSoulIOConfigBuilder::new()) } @@ -61,56 +75,60 @@ pub(crate) fn parse_table_info_partitions(partitions: String) -> (Vec, V let (range_keys, hash_keys) = partitions.split_at(partitions.find(';').unwrap()); let hash_keys = &hash_keys[1..]; ( - range_keys.split(',') + range_keys + .split(',') .collect::>() .iter() - .filter_map(|str| if str.is_empty() { - None - } else { - Some(str.to_string()) - }) - .collect::>(), - hash_keys.split(',') + .filter_map(|str| if str.is_empty() { None } else { Some(str.to_string()) }) + .collect::>(), + hash_keys + .split(',') .collect::>() .iter() - .filter_map(|str| if str.is_empty() { - None - } else { - Some(str.to_string()) - }) - .collect::>() + .filter_map(|str| if str.is_empty() { None } else { Some(str.to_string()) }) + .collect::>(), ) } -pub(crate) async fn commit_data(client: MetaDataClientRef, table_name: &str, partitions: Vec<(String, String)>, files: &[String]) -> Result<()>{ +pub(crate) async fn commit_data( + client: MetaDataClientRef, + table_name: &str, + partitions: Vec<(String, String)>, + files: &[String], +) -> Result<()> { let table_name_id = client.get_table_name_id_by_table_name(table_name, "default").await?; - client.commit_data_commit_info(DataCommitInfo { - table_id: table_name_id.table_id, - partition_desc: if partitions.is_empty() { - "-5".to_string() - } else { - partitions + client + .commit_data_commit_info(DataCommitInfo { + table_id: table_name_id.table_id, + partition_desc: if partitions.is_empty() { + "-5".to_string() + } else { + partitions + .iter() + .map(|(k, v)| format!("{}={}", k, v)) + .collect::>() + .join(",") + }, + file_ops: files .iter() - .map(|(k, v)| format!("{}={}", k, v)) - .collect::>() - .join(",") - }, - file_ops: files - .iter() - .map(|file| DataFileOp { - file_op: FileOp::Add as i32, - path: file.clone(), - ..Default::default() - }) - .collect(), - commit_op: CommitOp::AppendCommit as i32, - timestamp: SystemTime::now().duration_since(SystemTime::UNIX_EPOCH).unwrap().as_secs() as i64, - commit_id: { - let (high, low) = uuid::Uuid::new_v4().as_u64_pair(); - Some(Uuid{high, low}) - }, - committed: false, - domain: "public".to_string(), - }).await?; + .map(|file| DataFileOp { + file_op: FileOp::Add as i32, + path: file.clone(), + ..Default::default() + }) + .collect(), + commit_op: CommitOp::AppendCommit as i32, + timestamp: SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs() as i64, + commit_id: { + let (high, low) = uuid::Uuid::new_v4().as_u64_pair(); + Some(Uuid { high, low }) + }, + committed: false, + domain: "public".to_string(), + }) + .await?; Ok(()) -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs b/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs index e97dd6320..852ab026f 100644 --- a/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs +++ b/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs @@ -1,53 +1,64 @@ +use std::any::Any; use std::collections::HashMap; use std::fmt::{self, Debug}; use std::sync::Arc; -use std::any::Any; -use arrow::array::{UInt64Array, ArrayRef}; +use arrow::array::{ArrayRef, UInt64Array}; use arrow::record_batch::RecordBatch; use async_trait::async_trait; -use arrow::datatypes::{SchemaRef, Schema, Field, DataType}; -use datafusion::common::{Statistics, FileType}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::common::{FileType, Statistics}; use datafusion::error::DataFusionError; use datafusion::execution::TaskContext; use datafusion::physical_expr::PhysicalSortExpr; use datafusion::physical_plan::common::AbortOnDropSingle; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; -use datafusion::physical_plan::{DisplayAs, DisplayFormatType, SendableRecordBatchStream, Partitioning, Distribution}; +use datafusion::physical_plan::{DisplayAs, DisplayFormatType, Distribution, Partitioning, SendableRecordBatchStream}; use datafusion::scalar::ScalarValue; -use datafusion::{datasource::{file_format::{parquet::ParquetFormat, FileFormat}, physical_plan::{FileSinkConfig, FileScanConfig}}, physical_plan::{ExecutionPlan, PhysicalExpr}, execution::context::SessionState, physical_expr::PhysicalSortRequirement, error::Result}; -use lakesoul_io::lakesoul_writer::{MultiPartAsyncWriter, AsyncBatchWriter}; -use lakesoul_metadata::{MetaDataClientRef, MetaDataClient}; -use object_store::{ObjectStore, ObjectMeta}; +use datafusion::{ + datasource::{ + file_format::{parquet::ParquetFormat, FileFormat}, + physical_plan::{FileScanConfig, FileSinkConfig}, + }, + error::Result, + execution::context::SessionState, + physical_expr::PhysicalSortRequirement, + physical_plan::{ExecutionPlan, PhysicalExpr}, +}; +use futures::StreamExt; +use lakesoul_io::lakesoul_writer::{AsyncBatchWriter, MultiPartAsyncWriter}; +use lakesoul_metadata::{MetaDataClient, MetaDataClientRef}; +use object_store::{ObjectMeta, ObjectStore}; use proto::proto::entity::TableInfo; use rand::distributions::DistString; -use futures::StreamExt; use tokio::sync::Mutex; use tokio::task::JoinHandle; - use crate::catalog::commit_data; use crate::lakesoul_table::helpers::{create_io_config_builder_from_table_info, get_columnar_value}; pub struct LakeSoulMetaDataParquetFormat { parquet_format: Arc, client: MetaDataClientRef, - table_info: Arc + table_info: Arc, } impl Debug for LakeSoulMetaDataParquetFormat { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("LakeSoulMetaDataParquetFormat") - .finish() + f.debug_struct("LakeSoulMetaDataParquetFormat").finish() } } impl LakeSoulMetaDataParquetFormat { pub async fn new(parquet_format: Arc, table_info: Arc) -> crate::error::Result { let client = Arc::new(MetaDataClient::from_env().await?); - Ok(Self { parquet_format, client, table_info }) + Ok(Self { + parquet_format, + client, + table_info, + }) } fn client(&self) -> MetaDataClientRef { @@ -61,7 +72,7 @@ impl LakeSoulMetaDataParquetFormat { #[async_trait] impl FileFormat for LakeSoulMetaDataParquetFormat { - fn as_any(&self) -> &dyn Any { + fn as_any(&self) -> &dyn Any { self } @@ -81,7 +92,9 @@ impl FileFormat for LakeSoulMetaDataParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { - self.parquet_format.infer_stats(state, store, table_schema, object).await + self.parquet_format + .infer_stats(state, store, table_schema, object) + .await } async fn create_physical_plan( @@ -101,39 +114,37 @@ impl FileFormat for LakeSoulMetaDataParquetFormat { order_requirements: Option>, ) -> Result> { if conf.overwrite { - return Err(DataFusionError::NotImplemented("Overwrites are not implemented yet for Parquet".to_string())); + return Err(DataFusionError::NotImplemented( + "Overwrites are not implemented yet for Parquet".to_string(), + )); } - Ok(Arc::new(LakeSoulHashSinkExec::new( - input, - order_requirements, - self.table_info(), - self.client() - ).await?) as _) + Ok( + Arc::new(LakeSoulHashSinkExec::new(input, order_requirements, self.table_info(), self.client()).await?) + as _, + ) } fn file_type(&self) -> FileType { FileType::PARQUET } - } - // /// Execution plan for writing record batches to a [`LakeSoulParquetSink`] // /// // /// Returns a single row with the number of values written pub struct LakeSoulHashSinkExec { /// Input plan that produces the record batches to be written. input: Arc, - + /// Schema describing the structure of the output data. count_schema: SchemaRef, /// Optional required sort order for output data. sort_order: Option>, - table_info: Arc, + table_info: Arc, - metadata_client: MetaDataClientRef, + metadata_client: MetaDataClientRef, } impl fmt::Debug for LakeSoulHashSinkExec { @@ -155,7 +166,7 @@ impl LakeSoulHashSinkExec { count_schema: make_count_schema(), sort_order, table_info, - metadata_client + metadata_client, }) } @@ -183,7 +194,7 @@ impl LakeSoulHashSinkExec { context: Arc, table_info: Arc, write_id: String, - partitioned_file_path_and_row_count : Arc, (Vec, u64)>>> + partitioned_file_path_and_row_count: Arc, (Vec, u64)>>>, ) -> Result { let mut data = input.execute(partition, context.clone())?; @@ -201,17 +212,21 @@ impl LakeSoulHashSinkExec { .with_schema(batch.schema()) .build(); - let writer = - MultiPartAsyncWriter::try_new_with_context(&mut config, context.clone()).await?; + let writer = MultiPartAsyncWriter::try_new_with_context(&mut config, context.clone()).await?; partitioned_writer.insert(columnar_value.clone(), Box::new(writer)); } - + if let Some(async_writer) = partitioned_writer.get_mut(&columnar_value) { - if let Some(file_path_and_row_count) = partitioned_file_path_and_row_count_locked.get_mut(&columnar_value) { + if let Some(file_path_and_row_count) = + partitioned_file_path_and_row_count_locked.get_mut(&columnar_value) + { file_path_and_row_count.0.push(file_absolute_path); file_path_and_row_count.1 += batch.num_rows() as u64; } else { - partitioned_file_path_and_row_count_locked.insert(columnar_value.clone(), (vec![file_absolute_path], batch.num_rows() as u64)); + partitioned_file_path_and_row_count_locked.insert( + columnar_value.clone(), + (vec![file_absolute_path], batch.num_rows() as u64), + ); } row_count += batch.num_rows(); async_writer.write_record_batch(batch).await?; @@ -227,21 +242,20 @@ impl LakeSoulHashSinkExec { } async fn wait_for_commit( - join_handles: Vec>>, + join_handles: Vec>>, client: MetaDataClientRef, table_name: String, - partitioned_file_path_and_row_count : Arc, (Vec, u64)>>> + partitioned_file_path_and_row_count: Arc, (Vec, u64)>>>, ) -> Result { - let count = futures::future::join_all(join_handles) - .await - .iter() - .try_fold(0u64, |counter, result| { - match &result { + let count = + futures::future::join_all(join_handles) + .await + .iter() + .try_fold(0u64, |counter, result| match &result { Ok(Ok(count)) => Ok(counter + count), Ok(Err(e)) => Err(DataFusionError::Execution(format!("{}", e))), - Err(e) => Err(DataFusionError::Execution(format!("{}", e))), - } - })?; + Err(e) => Err(DataFusionError::Execution(format!("{}", e))), + })?; let partitioned_file_path_and_row_count = partitioned_file_path_and_row_count.lock().await; for (columnar_value, (files, _)) in partitioned_file_path_and_row_count.iter() { @@ -249,19 +263,16 @@ impl LakeSoulHashSinkExec { .iter() .map(|(column, value)| (column.to_string(), value.to_string())) .collect::>(); - commit_data(client.clone(), &table_name, partition_desc, &files).await.map_err(|e| DataFusionError::External(Box::new(e)))?; + commit_data(client.clone(), &table_name, partition_desc, &files) + .await + .map_err(|e| DataFusionError::External(Box::new(e)))?; } Ok(count) } - } impl DisplayAs for LakeSoulHashSinkExec { - fn fmt_as( - &self, - _t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "LakeSoulHashSinkExec") } } @@ -322,10 +333,7 @@ impl ExecutionPlan for LakeSoulHashSinkExec { vec![self.input.clone()] } - fn with_new_children( - self: Arc, - children: Vec>, - ) -> Result> { + fn with_new_children(self: Arc, children: Vec>) -> Result> { Ok(Arc::new(Self { input: children[0].clone(), count_schema: self.count_schema.clone(), @@ -341,74 +349,66 @@ impl ExecutionPlan for LakeSoulHashSinkExec { /// Execute the plan and return a stream of `RecordBatch`es for /// the specified partition. - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { + fn execute(&self, partition: usize, context: Arc) -> Result { if partition != 0 { - return Err(DataFusionError::NotImplemented(format!("FileSinkExec can only be called on partition 0!"))); + return Err(DataFusionError::NotImplemented(format!( + "FileSinkExec can only be called on partition 0!" + ))); } let num_input_partitions = self.input.output_partitioning().partition_count(); // launch one async task per *input* partition let mut join_handles = vec![]; - let write_id = - rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); + let write_id = rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - let partitioned_file_path_and_row_count = Arc::new(Mutex::new(HashMap::, (Vec, u64)>::new())); + let partitioned_file_path_and_row_count = Arc::new(Mutex::new(HashMap::< + Vec<(String, ScalarValue)>, + (Vec, u64), + >::new())); for i in 0..num_input_partitions { - let sink_task = - tokio::spawn(Self::pull_and_sink( - self.input().clone(), - i, - context.clone(), - self.table_info(), - write_id.clone(), - partitioned_file_path_and_row_count.clone() + let sink_task = tokio::spawn(Self::pull_and_sink( + self.input().clone(), + i, + context.clone(), + self.table_info(), + write_id.clone(), + partitioned_file_path_and_row_count.clone(), )); // // In a separate task, wait for each input to be done // // (and pass along any errors, including panic!s) join_handles.push(sink_task); } - let join_handle = AbortOnDropSingle::new( - tokio::spawn(Self::wait_for_commit( + let join_handle = AbortOnDropSingle::new(tokio::spawn(Self::wait_for_commit( join_handles, self.metadata_client(), self.table_info().table_name.clone(), - partitioned_file_path_and_row_count + partitioned_file_path_and_row_count, ))); + // }); - - // }); - // let abort_helper = Arc::new(AbortOnDropMany(join_handles)); let count_schema = self.count_schema.clone(); // let count = futures::future::join_all(join_handles).await; - // for (columnar_values, result) in partitioned_file_path_and_row_count.lock().await.iter() { - // match commit_data(self.metadata_client(), self.table_info().table_name.as_str(), &result.0).await { - // Ok(()) => todo!(), - // Err(_) => todo!(), - // } - // } - + // for (columnar_values, result) in partitioned_file_path_and_row_count.lock().await.iter() { + // match commit_data(self.metadata_client(), self.table_info().table_name.as_str(), &result.0).await { + // Ok(()) => todo!(), + // Err(_) => todo!(), + // } + // } let stream = futures::stream::once(async move { match join_handle.await { Ok(Ok(count)) => Ok(make_count_batch(count)), - other => Ok(make_count_batch(u64::MAX)) + other => Ok(make_count_batch(u64::MAX)), } }) .boxed(); - Ok(Box::pin(RecordBatchStreamAdapter::new( - count_schema, - stream, - ))) + Ok(Box::pin(RecordBatchStreamAdapter::new(count_schema, stream))) } } @@ -429,9 +429,5 @@ fn make_count_batch(count: u64) -> RecordBatch { fn make_count_schema() -> SchemaRef { // define a schema. - Arc::new(Schema::new(vec![Field::new( - "count", - DataType::UInt64, - false, - )])) + Arc::new(Schema::new(vec![Field::new("count", DataType::UInt64, false)])) } diff --git a/rust/lakesoul-datafusion/src/datasource/mod.rs b/rust/lakesoul-datafusion/src/datasource/mod.rs index 3072d60b0..39c68e496 100644 --- a/rust/lakesoul-datafusion/src/datasource/mod.rs +++ b/rust/lakesoul-datafusion/src/datasource/mod.rs @@ -2,5 +2,5 @@ // // SPDX-License-Identifier: Apache-2.0 -pub mod table_provider; pub mod file_format; +pub mod table_provider; diff --git a/rust/lakesoul-datafusion/src/datasource/table_provider.rs b/rust/lakesoul-datafusion/src/datasource/table_provider.rs index 0fb854d36..090b6e26c 100644 --- a/rust/lakesoul-datafusion/src/datasource/table_provider.rs +++ b/rust/lakesoul-datafusion/src/datasource/table_provider.rs @@ -2,18 +2,18 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::sync::Arc; use std::any::Any; +use std::sync::Arc; use arrow::datatypes::SchemaRef; use async_trait::async_trait; -use datafusion::datasource::TableProvider; -use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::file_format::FileFormat; +use datafusion::datasource::TableProvider; use datafusion::error::Result; -use datafusion::logical_expr::{TableType, TableProviderFilterPushDown}; +use datafusion::logical_expr::{TableProviderFilterPushDown, TableType}; use datafusion::physical_plan::ExecutionPlan; use datafusion::{execution::context::SessionState, logical_expr::Expr}; @@ -28,7 +28,6 @@ use crate::serialize::arrow_java::schema_from_metadata_str; use super::file_format::LakeSoulMetaDataParquetFormat; - /// Reads data from LakeSoul /// /// # Features @@ -43,7 +42,7 @@ use super::file_format::LakeSoulMetaDataParquetFormat; /// /// ``` pub struct LakeSoulTableProvider { - listing_table: Arc, + listing_table: Arc, table_info: Arc, schema: SchemaRef, primary_keys: Vec, @@ -54,33 +53,36 @@ impl LakeSoulTableProvider { session_state: &SessionState, lakesoul_io_config: LakeSoulIOConfig, table_info: Arc, - as_sink: bool + as_sink: bool, ) -> crate::error::Result { let schema = schema_from_metadata_str(&table_info.table_schema); let (_, hash_partitions) = parse_table_info_partitions(table_info.partitions.clone()); let file_format: Arc = match as_sink { - true => Arc::new(LakeSoulMetaDataParquetFormat::new( - Arc::new(ParquetFormat::new()), - table_info.clone() - ).await?), + true => { + Arc::new(LakeSoulMetaDataParquetFormat::new(Arc::new(ParquetFormat::new()), table_info.clone()).await?) + } false => Arc::new(LakeSoulParquetFormat::new( - Arc::new(ParquetFormat::new()), - lakesoul_io_config.clone())) + Arc::new(ParquetFormat::new()), + lakesoul_io_config.clone(), + )), }; Ok(Self { - listing_table: Arc::new(LakeSoulListingTable::new_with_config_and_format( - session_state, - lakesoul_io_config, - file_format, - as_sink - ).await?), + listing_table: Arc::new( + LakeSoulListingTable::new_with_config_and_format( + session_state, + lakesoul_io_config, + file_format, + as_sink, + ) + .await?, + ), table_info, schema, primary_keys: hash_partitions, }) } - + fn primary_keys(&self) -> &[String] { &self.primary_keys } @@ -90,7 +92,6 @@ impl LakeSoulTableProvider { } } - #[async_trait] impl TableProvider for LakeSoulTableProvider { fn as_any(&self) -> &dyn Any { @@ -115,10 +116,7 @@ impl TableProvider for LakeSoulTableProvider { self.listing_table.scan(state, projection, filters, limit).await } - fn supports_filters_pushdown( - &self, - filters: &[&Expr], - ) -> Result> { + fn supports_filters_pushdown(&self, filters: &[&Expr]) -> Result> { self.listing_table.supports_filters_pushdown(filters) } diff --git a/rust/lakesoul-datafusion/src/error.rs b/rust/lakesoul-datafusion/src/error.rs index 2bcd6336c..89fd60069 100644 --- a/rust/lakesoul-datafusion/src/error.rs +++ b/rust/lakesoul-datafusion/src/error.rs @@ -2,12 +2,11 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::{error::Error, sync::Arc, result, fmt::Display}; +use std::{error::Error, fmt::Display, result, sync::Arc}; -use lakesoul_io::lakesoul_reader::{DataFusionError, ArrowError}; +use lakesoul_io::lakesoul_reader::{ArrowError, DataFusionError}; use lakesoul_metadata::error::LakeSoulMetaDataError; - /// Result type for operations that could result in an [LakeSoulMetaDataError] pub type Result = result::Result; @@ -58,8 +57,11 @@ impl Display for LakeSoulError { LakeSoulError::SerdeJsonError(ref desc) => write!(f, "serde_json error: {desc}"), LakeSoulError::ArrowError(ref desc) => write!(f, "arrow error: {desc}"), LakeSoulError::Internal(ref desc) => { - write!(f, "Internal error: {desc}.\nThis was likely caused by a bug in LakeSoul's \ - code and we would welcome that you file an bug report in our issue tracker") + write!( + f, + "Internal error: {desc}.\nThis was likely caused by a bug in LakeSoul's \ + code and we would welcome that you file an bug report in our issue tracker" + ) } } } @@ -75,4 +77,4 @@ impl Error for LakeSoulError { LakeSoulError::Internal(_) => None, } } -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs b/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs index c64bd8bda..601966ea4 100644 --- a/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs +++ b/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs @@ -11,7 +11,10 @@ use datafusion::scalar::ScalarValue; use lakesoul_io::lakesoul_io_config::LakeSoulIOConfigBuilder; use proto::proto::entity::TableInfo; -use crate::{serialize::arrow_java::schema_from_metadata_str, catalog::{parse_table_info_partitions, LakeSoulTableProperty}}; +use crate::{ + catalog::{parse_table_info_partitions, LakeSoulTableProperty}, + serialize::arrow_java::schema_from_metadata_str, +}; pub(crate) fn create_io_config_builder_from_table_info(table_info: Arc) -> LakeSoulIOConfigBuilder { let (range_partitions, hash_partitions) = parse_table_info_partitions(table_info.partitions.clone()); @@ -19,21 +22,11 @@ pub(crate) fn create_io_config_builder_from_table_info(table_info: Arc Vec<(String, ScalarValue)> { +pub fn get_columnar_value(batch: &RecordBatch) -> Vec<(String, ScalarValue)> { vec![] -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs b/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs index bb41f4a9c..9d511863d 100644 --- a/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs +++ b/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs @@ -4,15 +4,25 @@ pub mod helpers; -use std::{sync::Arc, ops::Deref}; - -use arrow::{util::pretty::print_batches, datatypes::SchemaRef}; -use lakesoul_io::{lakesoul_reader::RecordBatch, lakesoul_io_config::create_session_context_with_planner}; -use datafusion::{dataframe::DataFrame, logical_expr::LogicalPlanBuilder, execution::context::{SessionContext, SessionState}, datasource::TableProvider}; +use std::{ops::Deref, sync::Arc}; + +use arrow::{datatypes::SchemaRef, util::pretty::print_batches}; +use datafusion::{ + dataframe::DataFrame, + datasource::TableProvider, + execution::context::{SessionContext, SessionState}, + logical_expr::LogicalPlanBuilder, +}; +use lakesoul_io::{lakesoul_io_config::create_session_context_with_planner, lakesoul_reader::RecordBatch}; use lakesoul_metadata::{MetaDataClient, MetaDataClientRef}; use proto::proto::entity::TableInfo; -use crate::{error::Result, catalog::{create_io_config_builder, parse_table_info_partitions, LakeSoulTableProperty}, planner::query_planner::LakeSoulQueryPlanner, serialize::arrow_java::schema_from_metadata_str}; +use crate::{ + catalog::{create_io_config_builder, parse_table_info_partitions, LakeSoulTableProperty}, + error::Result, + planner::query_planner::LakeSoulQueryPlanner, + serialize::arrow_java::schema_from_metadata_str, +}; use crate::datasource::table_provider::LakeSoulTableProvider; @@ -48,39 +58,37 @@ impl LakeSoulTable { pub async fn new_with_client_and_table_info(client: MetaDataClientRef, table_info: TableInfo) -> Result { let table_schema = schema_from_metadata_str(&table_info.table_schema); - + let table_name = table_info.table_name.clone(); let properties = serde_json::from_str::(&table_info.properties).unwrap(); let (_, hash_partitions) = parse_table_info_partitions(table_info.partitions.clone()); - - Ok(Self { + Ok(Self { client, table_info: Arc::new(table_info), - table_name, + table_name, table_schema, primary_keys: hash_partitions, - properties + properties, }) } pub async fn execute_upsert(&self, record_batch: RecordBatch) -> Result<()> { let client = Arc::new(MetaDataClient::from_env().await?); let builder = create_io_config_builder(client, None, false).await?; - let sess_ctx = create_session_context_with_planner( - &mut builder.clone().build(), - Some(LakeSoulQueryPlanner::new_ref()) - )?; - + let sess_ctx = + create_session_context_with_planner(&mut builder.clone().build(), Some(LakeSoulQueryPlanner::new_ref()))?; + let schema = record_batch.schema(); let logical_plan = LogicalPlanBuilder::insert_into( - sess_ctx.read_batch(record_batch)?.into_unoptimized_plan(), - self.table_name().to_string(), + sess_ctx.read_batch(record_batch)?.into_unoptimized_plan(), + self.table_name().to_string(), schema.deref(), - false)? - .build()?; + false, + )? + .build()?; let dataframe = DataFrame::new(sess_ctx.state(), logical_plan); - + let results = dataframe // .explain(true, false)? .collect() @@ -88,13 +96,13 @@ impl LakeSoulTable { Ok(()) // Ok(print_batches(&results)?) - } pub async fn to_dataframe(&self, context: &SessionContext) -> Result { - let config_builder = create_io_config_builder(self.client(), Some(self.table_name()), true) - .await?; - let provider = Arc::new(LakeSoulTableProvider::try_new(&context.state(), config_builder.build(), self.table_info(), false).await?); + let config_builder = create_io_config_builder(self.client(), Some(self.table_name()), true).await?; + let provider = Arc::new( + LakeSoulTableProvider::try_new(&context.state(), config_builder.build(), self.table_info(), false).await?, + ); Ok(context.read_table(provider)?) } @@ -102,7 +110,9 @@ impl LakeSoulTable { let config_builder = create_io_config_builder(self.client(), Some(self.table_name()), false) .await? .with_prefix(self.table_info.table_path.clone()); - Ok(Arc::new(LakeSoulTableProvider::try_new(session_state, config_builder.build(), self.table_info(), true).await?)) + Ok(Arc::new( + LakeSoulTableProvider::try_new(session_state, config_builder.build(), self.table_info(), true).await?, + )) } pub fn table_name(&self) -> &str { @@ -124,12 +134,11 @@ impl LakeSoulTable { pub fn hash_bucket_num(&self) -> usize { match self.properties.hash_bucket_num { Some(hash_bucket_num) => hash_bucket_num, - None => 1 + None => 1, } } pub fn schema(&self) -> SchemaRef { self.table_schema.clone() } - } diff --git a/rust/lakesoul-datafusion/src/lib.rs b/rust/lakesoul-datafusion/src/lib.rs index a0c84f170..102580e16 100644 --- a/rust/lakesoul-datafusion/src/lib.rs +++ b/rust/lakesoul-datafusion/src/lib.rs @@ -2,13 +2,12 @@ // // SPDX-License-Identifier: Apache-2.0 - mod catalog; +mod datasource; mod error; -mod serialize; -mod planner; mod lakesoul_table; -mod datasource; +mod planner; +mod serialize; #[cfg(test)] -mod test; \ No newline at end of file +mod test; diff --git a/rust/lakesoul-datafusion/src/planner/mod.rs b/rust/lakesoul-datafusion/src/planner/mod.rs index 3881e11b0..ba943b645 100644 --- a/rust/lakesoul-datafusion/src/planner/mod.rs +++ b/rust/lakesoul-datafusion/src/planner/mod.rs @@ -2,5 +2,5 @@ // // SPDX-License-Identifier: Apache-2.0 +mod physical_planner; pub mod query_planner; -mod physical_planner; \ No newline at end of file diff --git a/rust/lakesoul-datafusion/src/planner/physical_planner.rs b/rust/lakesoul-datafusion/src/planner/physical_planner.rs index 63b79e964..f0d0b9c75 100644 --- a/rust/lakesoul-datafusion/src/planner/physical_planner.rs +++ b/rust/lakesoul-datafusion/src/planner/physical_planner.rs @@ -7,28 +7,26 @@ use std::sync::Arc; use arrow::datatypes::Schema; - use datafusion::common::{DFSchema, SchemaExt}; +use datafusion::error::{DataFusionError, Result}; +use datafusion::execution::context::SessionState; +use datafusion::logical_expr::{Expr, LogicalPlan, Partitioning as LogicalPartitioning, Repartition, Sort}; +use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; -use datafusion::physical_planner::{PhysicalPlanner, DefaultPhysicalPlanner, create_physical_sort_expr}; use datafusion::physical_plan::{ExecutionPlan, Partitioning}; -use datafusion::physical_expr::PhysicalExpr; -use datafusion::error::{Result, DataFusionError}; -use datafusion::logical_expr::{Expr, LogicalPlan, Partitioning as LogicalPartitioning, Sort, Repartition}; -use datafusion::execution::context::SessionState; +use datafusion::physical_planner::{create_physical_sort_expr, DefaultPhysicalPlanner, PhysicalPlanner}; use async_trait::async_trait; -use datafusion::logical_expr::{DmlStatement, WriteOp, LogicalPlanBuilder}; -use lakesoul_io::helpers::{create_sort_exprs, create_hash_partitioning}; +use datafusion::logical_expr::{DmlStatement, LogicalPlanBuilder, WriteOp}; +use lakesoul_io::helpers::{create_hash_partitioning, create_sort_exprs}; use lakesoul_io::repartition::RepartitionByRangeAndHashExec; use crate::lakesoul_table::LakeSoulTable; - pub struct LakeSoulPhysicalPlanner { - default_planner : DefaultPhysicalPlanner, + default_planner: DefaultPhysicalPlanner, } impl LakeSoulPhysicalPlanner { @@ -60,7 +58,7 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { match lakesoul_table.as_sink_provider(session_state).await { Ok(provider) => { let physical_input = self.create_physical_plan(&input, session_state).await?; - + let physical_input = if lakesoul_table.primary_keys().is_empty() { if !lakesoul_table .schema() @@ -68,7 +66,7 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { { return Err(DataFusionError::Plan( // Return an error if schema of the input query does not match with the table schema. - format!("Inserting query must have the same schema with the table.") + format!("Inserting query must have the same schema with the table."), )); } physical_input @@ -76,10 +74,10 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { let input_schema = physical_input.schema(); let input_dfschema = input.as_ref().schema(); let sort_expr = create_sort_exprs( - &lakesoul_table.primary_keys(), - input_dfschema, - &input_schema, - session_state + &lakesoul_table.primary_keys(), + input_dfschema, + &input_schema, + session_state, )?; let hash_partitioning = create_hash_partitioning( &lakesoul_table.primary_keys(), @@ -93,19 +91,22 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { }; provider.insert_into(session_state, physical_input, false).await - } - Err(e) => - return Err(DataFusionError::External(Box::new(e))) + } + Err(e) => return Err(DataFusionError::External(Box::new(e))), } } LogicalPlan::Statement(statement) => { // DataFusion is a read-only query engine, but also a library, so consumers may implement this let name = statement.name(); - Err(DataFusionError::NotImplemented( - format!("Unsupported logical plan: Statement({name})") - )) + Err(DataFusionError::NotImplemented(format!( + "Unsupported logical plan: Statement({name})" + ))) + } + _ => { + self.default_planner + .create_physical_plan(logical_plan, session_state) + .await } - _ => self.default_planner.create_physical_plan(logical_plan, session_state).await } } @@ -125,7 +126,9 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { session_state: &SessionState, ) -> Result> { match expr { - _ => self.default_planner.create_physical_expr(expr, input_dfschema, input_schema, session_state) + _ => self + .default_planner + .create_physical_expr(expr, input_dfschema, input_schema, session_state), } } -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/planner/query_planner.rs b/rust/lakesoul-datafusion/src/planner/query_planner.rs index 2efce4a9c..61c7e97b0 100644 --- a/rust/lakesoul-datafusion/src/planner/query_planner.rs +++ b/rust/lakesoul-datafusion/src/planner/query_planner.rs @@ -4,10 +4,10 @@ use std::sync::Arc; -use datafusion::execution::context::{QueryPlanner, SessionState}; -use datafusion::physical_plan::ExecutionPlan; use datafusion::error::Result; +use datafusion::execution::context::{QueryPlanner, SessionState}; use datafusion::logical_expr::LogicalPlan; +use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_planner::PhysicalPlanner; use crate::planner::physical_planner::LakeSoulPhysicalPlanner; @@ -20,7 +20,7 @@ pub struct LakeSoulQueryPlanner {} impl LakeSoulQueryPlanner { pub fn new_ref() -> Arc { - Arc::new(Self{}) + Arc::new(Self {}) } } @@ -33,9 +33,6 @@ impl QueryPlanner for LakeSoulQueryPlanner { session_state: &SessionState, ) -> Result> { let planner = LakeSoulPhysicalPlanner::new(); - planner - .create_physical_plan(logical_plan, session_state) - .await + planner.create_physical_plan(logical_plan, session_state).await } - -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/serialize/arrow_java.rs b/rust/lakesoul-datafusion/src/serialize/arrow_java.rs index 1f4e05da9..3d7160282 100644 --- a/rust/lakesoul-datafusion/src/serialize/arrow_java.rs +++ b/rust/lakesoul-datafusion/src/serialize/arrow_java.rs @@ -4,11 +4,11 @@ use std::{collections::HashMap, sync::Arc}; -use arrow::datatypes::{FieldRef, Field, Fields, DataType, TimeUnit, SchemaRef, Schema}; +use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef, TimeUnit}; #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] #[serde(tag = "name")] -enum ArrowJavaType { +enum ArrowJavaType { #[serde(rename = "null")] Null, #[serde(rename = "struct")] @@ -20,14 +20,14 @@ enum ArrowJavaType { #[serde(rename = "fixedsizelist")] FixedSizeList { #[serde(rename = "listSize")] - list_size: i32 + list_size: i32, }, #[serde(rename = "union")] Union, #[serde(rename = "map")] Map { #[serde(rename = "keysSorted")] - keys_sorted: bool + keys_sorted: bool, }, #[serde(rename = "int")] Int { @@ -37,9 +37,7 @@ enum ArrowJavaType { bit_width: i32, }, #[serde(rename = "floatingpoint")] - FloatingPoint { - precision: String, - }, + FloatingPoint { precision: String }, #[serde(rename = "utf8")] Utf8, #[serde(rename = "largeutf8")] @@ -63,9 +61,7 @@ enum ArrowJavaType { bit_width: i32, }, #[serde(rename = "date")] - Date { - unit: String, - }, + Date { unit: String }, #[serde(rename = "time")] Time { #[serde(rename = "bitWidth")] @@ -73,19 +69,15 @@ enum ArrowJavaType { unit: String, }, #[serde(rename = "timestamp")] - Timestamp { - unit: String, - timezone: Option, - }, + Timestamp { unit: String, timezone: Option }, #[serde(rename = "interval")] Interval, #[serde(rename = "duration")] Duration, } - #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] -struct ArrowJavaField{ +struct ArrowJavaField { name: String, #[serde(rename = "type")] data_type: ArrowJavaType, @@ -104,29 +96,102 @@ impl From<&FieldRef> for ArrowJavaField { fn from(field: &FieldRef) -> Self { let name = field.name().clone(); let (data_type, children) = match field.data_type() { - DataType::Null => (ArrowJavaType::Null, vec![]), + DataType::Null => (ArrowJavaType::Null, vec![]), - DataType::Struct(fields) => (ArrowJavaType::Struct, fields.iter().map(ArrowJavaField::from).collect::>()), + DataType::Struct(fields) => ( + ArrowJavaType::Struct, + fields.iter().map(ArrowJavaField::from).collect::>(), + ), DataType::List(field) => (ArrowJavaType::List, vec![ArrowJavaField::from(field)]), DataType::LargeList(field) => (ArrowJavaType::LargeList, vec![ArrowJavaField::from(field)]), - DataType::FixedSizeList(field, list_size ) => (ArrowJavaType::FixedSizeList { list_size: *list_size }, vec![ArrowJavaField::from(field)]), + DataType::FixedSizeList(field, list_size) => ( + ArrowJavaType::FixedSizeList { list_size: *list_size }, + vec![ArrowJavaField::from(field)], + ), + DataType::Map(struct_field, key_sorted) => ( + ArrowJavaType::Map { + keys_sorted: *key_sorted, + }, + vec![ArrowJavaField::from(struct_field)], + ), - DataType::Map(struct_field, key_sorted) => (ArrowJavaType::Map { keys_sorted: *key_sorted }, vec![ArrowJavaField::from(struct_field)]), + DataType::Int8 => ( + ArrowJavaType::Int { + is_signed: true, + bit_width: 8, + }, + vec![], + ), + DataType::Int16 => ( + ArrowJavaType::Int { + is_signed: true, + bit_width: 16, + }, + vec![], + ), + DataType::Int32 => ( + ArrowJavaType::Int { + is_signed: true, + bit_width: 32, + }, + vec![], + ), + DataType::Int64 => ( + ArrowJavaType::Int { + is_signed: true, + bit_width: 64, + }, + vec![], + ), + DataType::UInt8 => ( + ArrowJavaType::Int { + is_signed: false, + bit_width: 8, + }, + vec![], + ), + DataType::UInt16 => ( + ArrowJavaType::Int { + is_signed: false, + bit_width: 16, + }, + vec![], + ), + DataType::UInt32 => ( + ArrowJavaType::Int { + is_signed: false, + bit_width: 32, + }, + vec![], + ), + DataType::UInt64 => ( + ArrowJavaType::Int { + is_signed: false, + bit_width: 64, + }, + vec![], + ), - DataType::Int8 => (ArrowJavaType::Int { is_signed: true, bit_width: 8 }, vec![]), - DataType::Int16 => (ArrowJavaType::Int { is_signed: true, bit_width: 16 }, vec![]), - DataType::Int32 => (ArrowJavaType::Int { is_signed: true, bit_width: 32 }, vec![]), - DataType::Int64 => (ArrowJavaType::Int { is_signed: true, bit_width: 64 }, vec![]), - DataType::UInt8 => (ArrowJavaType::Int { is_signed: false, bit_width: 8 }, vec![]), - DataType::UInt16 => (ArrowJavaType::Int { is_signed: false, bit_width: 16 }, vec![]), - DataType::UInt32 => (ArrowJavaType::Int { is_signed: false, bit_width: 32 }, vec![]), - DataType::UInt64 => (ArrowJavaType::Int { is_signed: false, bit_width: 64 }, vec![]), - - DataType::Float16 => (ArrowJavaType::FloatingPoint { precision: "HALF".to_string() }, vec![]), - DataType::Float32 => (ArrowJavaType::FloatingPoint { precision: "SINGLE".to_string() }, vec![]), - DataType::Float64 => (ArrowJavaType::FloatingPoint { precision: "DOUBLE".to_string() }, vec![]), + DataType::Float16 => ( + ArrowJavaType::FloatingPoint { + precision: "HALF".to_string(), + }, + vec![], + ), + DataType::Float32 => ( + ArrowJavaType::FloatingPoint { + precision: "SINGLE".to_string(), + }, + vec![], + ), + DataType::Float64 => ( + ArrowJavaType::FloatingPoint { + precision: "DOUBLE".to_string(), + }, + vec![], + ), DataType::Utf8 => (ArrowJavaType::Utf8, vec![]), DataType::LargeUtf8 => (ArrowJavaType::LargeUtf8, vec![]), @@ -137,61 +202,85 @@ impl From<&FieldRef> for ArrowJavaField { DataType::Boolean => (ArrowJavaType::Bool, vec![]), - DataType::Decimal128(precision, scale) => (ArrowJavaType::Decimal { precision: *precision, scale: *scale, bit_width: 128 }, vec![]), - DataType::Decimal256(precision, scale) => (ArrowJavaType::Decimal { precision: *precision, scale: *scale, bit_width: 256 }, vec![]), - - DataType::Date32 => (ArrowJavaType::Date { unit: "DAY".to_string() }, vec![]), - DataType::Date64 => (ArrowJavaType::Date { unit: "MILLISECOND".to_string() }, vec![]), + DataType::Decimal128(precision, scale) => ( + ArrowJavaType::Decimal { + precision: *precision, + scale: *scale, + bit_width: 128, + }, + vec![], + ), + DataType::Decimal256(precision, scale) => ( + ArrowJavaType::Decimal { + precision: *precision, + scale: *scale, + bit_width: 256, + }, + vec![], + ), + + DataType::Date32 => ( + ArrowJavaType::Date { + unit: "DAY".to_string(), + }, + vec![], + ), + DataType::Date64 => ( + ArrowJavaType::Date { + unit: "MILLISECOND".to_string(), + }, + vec![], + ), - DataType::Time32(unit) => - ( - ArrowJavaType::Time { - bit_width: 32, + DataType::Time32(unit) => ( + ArrowJavaType::Time { + bit_width: 32, unit: match unit { TimeUnit::Second => "SECOND".to_string(), TimeUnit::Microsecond => "MICROSECOND".to_string(), TimeUnit::Millisecond => "MILLISECOND".to_string(), TimeUnit::Nanosecond => "NANOSECOND".to_string(), - } }, - vec![] - ), - DataType::Time64(unit) => - ( - ArrowJavaType::Time { - bit_width: 64, + }, + }, + vec![], + ), + DataType::Time64(unit) => ( + ArrowJavaType::Time { + bit_width: 64, unit: match unit { TimeUnit::Second => "SECOND".to_string(), TimeUnit::Microsecond => "MICROSECOND".to_string(), TimeUnit::Millisecond => "MILLISECOND".to_string(), TimeUnit::Nanosecond => "NANOSECOND".to_string(), - } }, - vec![] - ), - DataType::Timestamp(unit, timezone) => - (ArrowJavaType::Timestamp { + }, + }, + vec![], + ), + DataType::Timestamp(unit, timezone) => ( + ArrowJavaType::Timestamp { unit: match unit { TimeUnit::Second => "SECOND".to_string(), TimeUnit::Microsecond => "MICROSECOND".to_string(), TimeUnit::Millisecond => "MILLISECOND".to_string(), TimeUnit::Nanosecond => "NANOSECOND".to_string(), - }, - timezone: timezone.as_ref().map(|s| s.to_string()) }, - vec![] - ), + timezone: timezone.as_ref().map(|s| s.to_string()), + }, + vec![], + ), DataType::Union(_, _) => todo!("Union type not supported"), DataType::Dictionary(_, _) => todo!("Dictionary type not supported"), DataType::Duration(_) => todo!("Duration type not supported"), DataType::Interval(_) => todo!("Interval type not supported"), - DataType::RunEndEncoded(_, _) => todo!("RunEndEncoded type not supported"), + DataType::RunEndEncoded(_, _) => todo!("RunEndEncoded type not supported"), }; let nullable = field.is_nullable(); - ArrowJavaField { - name, - data_type, - nullable, - children + ArrowJavaField { + name, + data_type, + nullable, + children, } } } @@ -201,15 +290,9 @@ impl From<&ArrowJavaField> for Field { let java_type = &field.data_type.clone(); let data_type = match java_type { ArrowJavaType::Null => DataType::Null, - ArrowJavaType::Struct => - DataType::Struct( - Fields::from(field - .children - .iter() - .map(|f| f.into()) - .collect::>() - ) - ), + ArrowJavaType::Struct => DataType::Struct(Fields::from( + field.children.iter().map(|f| f.into()).collect::>(), + )), ArrowJavaType::List => { assert!(field.children.len() == 1); DataType::List(Arc::new(field.children.first().unwrap().into())) @@ -223,7 +306,7 @@ impl From<&ArrowJavaField> for Field { DataType::FixedSizeList(Arc::new(field.children.first().unwrap().into()), *list_size) } ArrowJavaType::Union => todo!("Union type not supported"), - ArrowJavaType::Map{keys_sorted} => { + ArrowJavaType::Map { keys_sorted } => { assert!(field.children.len() == 1); DataType::Map(Arc::new(field.children.first().unwrap().into()), *keys_sorted) } @@ -246,21 +329,28 @@ impl From<&ArrowJavaField> for Field { } } } - ArrowJavaType::FloatingPoint { precision } => - match precision.as_str() { - "HALF" => DataType::Float16, - "SINGLE" => DataType::Float32, - "DOUBLE" => DataType::Float64, - other => panic!("FloatingPoint has an invalid precision = {}", other), - } + ArrowJavaType::FloatingPoint { precision } => match precision.as_str() { + "HALF" => DataType::Float16, + "SINGLE" => DataType::Float32, + "DOUBLE" => DataType::Float64, + other => panic!("FloatingPoint has an invalid precision = {}", other), + }, ArrowJavaType::Utf8 => DataType::Utf8, ArrowJavaType::LargeUtf8 => DataType::LargeUtf8, ArrowJavaType::Binary => DataType::Binary, ArrowJavaType::LargeBinary => DataType::LargeBinary, ArrowJavaType::FixedSizeBinary { bit_width } => DataType::FixedSizeBinary(*bit_width), ArrowJavaType::Bool => DataType::Boolean, - ArrowJavaType::Decimal { precision, scale, bit_width } if *bit_width > 128 => DataType::Decimal256(*precision, *scale), - ArrowJavaType::Decimal { precision, scale, bit_width } => DataType::Decimal128(*precision, *scale), + ArrowJavaType::Decimal { + precision, + scale, + bit_width, + } if *bit_width > 128 => DataType::Decimal256(*precision, *scale), + ArrowJavaType::Decimal { + precision, + scale, + bit_width, + } => DataType::Decimal128(*precision, *scale), ArrowJavaType::Date { unit } if unit == "DAY" => DataType::Date32, ArrowJavaType::Date { unit } => DataType::Date64, ArrowJavaType::Time { bit_width, unit } => { @@ -287,23 +377,19 @@ impl From<&ArrowJavaField> for Field { }; let timezone: Option> = timezone.as_ref().map(|t| Arc::from(t.as_str())); DataType::Timestamp(time_unit, timezone) - }, + } ArrowJavaType::Interval => todo!("Interval type not supported"), ArrowJavaType::Duration => todo!("Duration type not supported"), }; - Field::new( - field.name.clone(), - data_type, - field.nullable - ) + Field::new(field.name.clone(), data_type, field.nullable) } } impl From for ArrowJavaSchema { fn from(schema: SchemaRef) -> Self { - Self { + Self { fields: schema.fields().iter().map(ArrowJavaField::from).collect::>(), - metadata: None + metadata: None, } } } @@ -311,18 +397,17 @@ impl From for ArrowJavaSchema { impl From for SchemaRef { fn from(schema: ArrowJavaSchema) -> Self { SchemaRef::new(Schema::new( - schema.fields - .iter() - .map(|f| f.into()) - .collect::>() - )) + schema.fields.iter().map(|f| f.into()).collect::>(), + )) } } - pub fn schema_from_metadata_str(s: &str) -> SchemaRef { - serde_json::from_str::(s).map_or_else( |_| { - let java_schema = serde_json::from_str::(s).unwrap(); - java_schema.into() - }, SchemaRef::new) + serde_json::from_str::(s).map_or_else( + |_| { + let java_schema = serde_json::from_str::(s).unwrap(); + java_schema.into() + }, + SchemaRef::new, + ) } diff --git a/rust/lakesoul-datafusion/src/serialize/mod.rs b/rust/lakesoul-datafusion/src/serialize/mod.rs index cd51f820c..5ad51f68c 100644 --- a/rust/lakesoul-datafusion/src/serialize/mod.rs +++ b/rust/lakesoul-datafusion/src/serialize/mod.rs @@ -2,4 +2,4 @@ // // SPDX-License-Identifier: Apache-2.0 -pub mod arrow_java; \ No newline at end of file +pub mod arrow_java; diff --git a/rust/lakesoul-datafusion/src/test/insert_tests.rs b/rust/lakesoul-datafusion/src/test/insert_tests.rs index 4d70d0d38..bbe7ca0af 100644 --- a/rust/lakesoul-datafusion/src/test/insert_tests.rs +++ b/rust/lakesoul-datafusion/src/test/insert_tests.rs @@ -6,52 +6,59 @@ mod insert_tests { use std::sync::Arc; use arrow::array::*; - use arrow::datatypes::{Int32Type, i256}; - use lakesoul_io::filter::parser::Parser; - use lakesoul_io::lakesoul_io_config::{LakeSoulIOConfigBuilder, create_session_context}; - use datafusion::assert_batches_eq; - use arrow::{record_batch::RecordBatch, - array::{Int32Array, ArrayRef}, - datatypes::{SchemaRef, Schema, Field, DataType} + use arrow::datatypes::{i256, Int32Type}; + use arrow::{ + array::{ArrayRef, Int32Array}, + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, }; + use datafusion::assert_batches_eq; + use lakesoul_io::filter::parser::Parser; + use lakesoul_io::lakesoul_io_config::{create_session_context, LakeSoulIOConfigBuilder}; use lakesoul_metadata::{MetaDataClient, MetaDataClientRef}; use crate::lakesoul_table::LakeSoulTable; - use crate::{error::Result, catalog::{create_table, create_io_config_builder}}; + use crate::{ + catalog::{create_io_config_builder, create_table}, + error::Result, + }; async fn init_table(client: MetaDataClientRef, schema: SchemaRef, table_name: &str) -> Result<()> { - let builder = LakeSoulIOConfigBuilder::new() - .with_schema(schema.clone()); - // .with_primary_keys(pks); + let builder = LakeSoulIOConfigBuilder::new().with_schema(schema.clone()); + // .with_primary_keys(pks); create_table(client, table_name, builder.build()).await } - async fn init_partitioned_table(client: MetaDataClientRef, schema: SchemaRef, table_name: &str, partition_key: Vec<&str>) -> Result<()> { + async fn init_partitioned_table( + client: MetaDataClientRef, + schema: SchemaRef, + table_name: &str, + partition_key: Vec<&str>, + ) -> Result<()> { // todo: partitioned table is replaced by primary key table currently let builder = LakeSoulIOConfigBuilder::new() - .with_schema(schema.clone()) - .with_primary_keys(partition_key.into_iter().map(String::from).collect()); + .with_schema(schema.clone()) + .with_primary_keys(partition_key.into_iter().map(String::from).collect()); create_table(client, table_name, builder.build()).await } - async fn do_insert(record_batch: RecordBatch, table_name: &str) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; lakesoul_table.execute_upsert(record_batch).await } async fn check_insert( - client: MetaDataClientRef, - table_name: &str, - selected_cols: Vec<&str>, - filters: Option, - expected: &[&str] + client: MetaDataClientRef, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + expected: &[&str], ) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; let builder = create_io_config_builder(client, None, false).await?; let sess_ctx = create_session_context(&mut builder.clone().build())?; - + let dataframe = lakesoul_table.to_dataframe(&sess_ctx).await?; let schema = SchemaRef::new(dataframe.schema().into()); @@ -67,12 +74,8 @@ mod insert_tests { dataframe.select_columns(&selected_cols)? }; + let result = dataframe.collect().await?; - - let result = dataframe - .collect() - .await?; - assert_batches_eq!(expected, &result); Ok(()) } @@ -82,7 +85,11 @@ mod insert_tests { .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } @@ -90,14 +97,8 @@ mod insert_tests { let table_name = "test_insert_into_append"; let client = Arc::new(MetaDataClient::from_env().await?); let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); - init_table( - client.clone(), - record_batch.schema(), - table_name, - ).await?; - do_insert( - record_batch, - table_name).await?; + init_table(client.clone(), record_batch.schema(), table_name).await?; + do_insert(record_batch, table_name).await?; check_insert( client.clone(), table_name, @@ -112,21 +113,16 @@ mod insert_tests { "| 3 | 3 |", "+----+------+", ], - ).await + ) + .await } async fn test_insert_into_append_by_position() -> Result<()> { let table_name = "test_insert_into_append_by_position"; let client = Arc::new(MetaDataClient::from_env().await?); let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); - init_table( - client.clone(), - record_batch.schema(), - table_name, - ).await?; - do_insert( - record_batch, - table_name,).await?; + init_table(client.clone(), record_batch.schema(), table_name).await?; + do_insert(record_batch, table_name).await?; check_insert( client.clone(), table_name, @@ -141,22 +137,16 @@ mod insert_tests { "| 3 | 3 |", "+------+----+", ], - ).await + ) + .await } async fn test_insert_into_append_partitioned_table() -> Result<()> { let table_name = "test_insert_into_append_partitioned_table"; let client = Arc::new(MetaDataClient::from_env().await?); let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); - init_partitioned_table( - client.clone(), - record_batch.schema(), - table_name, - vec!["id"] - ).await?; - do_insert( - record_batch, - table_name,).await?; + init_partitioned_table(client.clone(), record_batch.schema(), table_name, vec!["id"]).await?; + do_insert(record_batch, table_name).await?; check_insert( client.clone(), table_name, @@ -171,21 +161,16 @@ mod insert_tests { "| 3 | 3 |", "+------+----+", ], - ).await + ) + .await } async fn test_insert_into_append_non_partitioned_table_and_read_with_filter() -> Result<()> { let table_name = "test_insert_into_append_non_partitioned_table_and_read_with_filter"; let client = Arc::new(MetaDataClient::from_env().await?); let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); - init_table( - client.clone(), - record_batch.schema(), - table_name, - ).await?; - do_insert( - record_batch, - table_name).await?; + init_table(client.clone(), record_batch.schema(), table_name).await?; + do_insert(record_batch, table_name).await?; check_insert( client.clone(), table_name, @@ -199,22 +184,16 @@ mod insert_tests { "| 2 | 2 |", "+------+----+", ], - ).await + ) + .await } async fn test_insert_into_append_partitioned_table_and_read_with_partition_filter() -> Result<()> { let table_name = "test_insert_into_append_partitioned_table_and_read_with_partition_filter"; let client = Arc::new(MetaDataClient::from_env().await?); let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); - init_partitioned_table( - client.clone(), - record_batch.schema(), - table_name, - vec!["id"] - ).await?; - do_insert( - record_batch, - table_name).await?; + init_partitioned_table(client.clone(), record_batch.schema(), table_name, vec!["id"]).await?; + do_insert(record_batch, table_name).await?; check_insert( client.clone(), table_name, @@ -228,27 +207,24 @@ mod insert_tests { "| 2 | 2 |", "+------+----+", ], - ).await + ) + .await } - + // todo: insert_overwrite is not supported by datafusion 27.0 // #[tokio::test] async fn test_insert_into_overwrite_non_partitioned_table() -> Result<()> { let table_name = "test_insert_into_overwrite_non_partitioned_table"; let client = Arc::new(MetaDataClient::from_env().await?); let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); - init_table( - client.clone(), - record_batch.schema(), - table_name, - ).await?; - do_insert( - record_batch, - table_name,).await?; - // todo: should do_insert_overwrite + init_table(client.clone(), record_batch.schema(), table_name).await?; + do_insert(record_batch, table_name).await?; + // todo: should do_insert_overwrite do_insert( create_batch_i32(vec!["id", "data"], vec![&[4, 5, 6], &[4, 5, 6]]), - table_name,).await?; + table_name, + ) + .await?; check_insert( client.clone(), table_name, @@ -263,7 +239,8 @@ mod insert_tests { "| 6 | 6 |", "+----+------+", ], - ).await + ) + .await } async fn test_insert_into_fails_when_missing_a_column() -> Result<()> { @@ -272,18 +249,24 @@ mod insert_tests { let record_batch = create_batch_i32(vec!["id", "data"], vec![&[1, 2, 3], &[1, 2, 3]]); init_table( client.clone(), - SchemaRef::new(Schema::new(["id", "data", "missing"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + SchemaRef::new(Schema::new( + ["id", "data", "missing"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), table_name, - ).await?; - match do_insert( - record_batch, - table_name).await { - Err(e) => { - dbg!(&e); - Ok(()) - } - Ok(()) => Err(crate::error::LakeSoulError::Internal("InsertInto should fail when missing columns".to_string())) + ) + .await?; + match do_insert(record_batch, table_name).await { + Err(e) => { + dbg!(&e); + Ok(()) } + Ok(()) => Err(crate::error::LakeSoulError::Internal( + "InsertInto should fail when missing columns".to_string(), + )), + } } async fn test_insert_into_fails_when_an_extra_column_is_present_but_can_evolve_schema() -> Result<()> { @@ -293,56 +276,92 @@ mod insert_tests { ("id", Arc::new(Int32Array::from(vec![1])) as ArrayRef, true), ("data", Arc::new(StringArray::from(vec!["a"])) as ArrayRef, true), ("fruit", Arc::new(StringArray::from(vec!["mango"])) as ArrayRef, true), - ]).unwrap(); + ]) + .unwrap(); init_table( client.clone(), - SchemaRef::new(Schema::new(["id", "data"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + SchemaRef::new(Schema::new( + ["id", "data"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), table_name, - ).await?; - match do_insert( - record_batch, - table_name).await { - Err(e) => { - dbg!(&e); - Ok(()) - } - Ok(()) => Err(crate::error::LakeSoulError::Internal("InsertInto should fails when an extra column is present but can evolve schema".to_string())) + ) + .await?; + match do_insert(record_batch, table_name).await { + Err(e) => { + dbg!(&e); + Ok(()) } + Ok(()) => Err(crate::error::LakeSoulError::Internal( + "InsertInto should fails when an extra column is present but can evolve schema".to_string(), + )), + } // todo: pass this case when SCHEMA_AUTO_MIGRATE is true } - async fn test_datatypes() -> Result<()>{ + async fn test_datatypes() -> Result<()> { let table_name = "test_datatypes"; let client = Arc::new(MetaDataClient::from_env().await?); // let mut client = MetaDataClient::from_config("host=127.0.0.1 port=5433 dbname=test_lakesoul_meta user=yugabyte password=yugabyte".to_string()); let iter = vec![ - ("Boolean", Arc::new(BooleanArray::from(vec![true, false])) as ArrayRef, true), - ("Binary", Arc::new(BinaryArray::from_vec(vec![&[1u8], &[2u8, 3u8]])) as ArrayRef, true), - + ( + "Boolean", + Arc::new(BooleanArray::from(vec![true, false])) as ArrayRef, + true, + ), + ( + "Binary", + Arc::new(BinaryArray::from_vec(vec![&[1u8], &[2u8, 3u8]])) as ArrayRef, + true, + ), ("Date32", Arc::new(Date32Array::from(vec![1, -2])) as ArrayRef, true), ("Date64", Arc::new(Date64Array::from(vec![1, -2])) as ArrayRef, true), - ("Decimal128", Arc::new(Decimal128Array::from(vec![1, -2])) as ArrayRef, true), - ("Decimal256", Arc::new(Decimal256Array::from(vec![Some(i256::default()), None])) as ArrayRef, true), - + ( + "Decimal128", + Arc::new(Decimal128Array::from(vec![1, -2])) as ArrayRef, + true, + ), + ( + "Decimal256", + Arc::new(Decimal256Array::from(vec![Some(i256::default()), None])) as ArrayRef, + true, + ), // ParquetError(ArrowError("Converting Duration to parquet not supported")) // ("DurationMicrosecond", Arc::new(DurationMicrosecondArray::from(vec![1])) as ArrayRef, true), // ("DurationMillisecond", Arc::new(DurationMillisecondArray::from(vec![1])) as ArrayRef, true), // ("Float16", Arc::new(Float16Array::from(vec![1.0])) as ArrayRef, true), - - ("FixedSizeBinary", Arc::new(FixedSizeBinaryArray::from(vec![&[1u8][..], &[2u8][..]])) as ArrayRef, true), - ("FixedSizeList", Arc::new(FixedSizeListArray::from_iter_primitive::(vec![ - Some(vec![Some(0), Some(1), Some(2)]), - None, - // Some(vec![Some(3), None, Some(5)]), - // Some(vec![Some(6), Some(7)]), - ], 3)) as ArrayRef, true), - - ("Float32", Arc::new(Float32Array::from(vec![1.0, -1.0])) as ArrayRef, true), - ("Float64", Arc::new(Float64Array::from(vec![1.0, -1.0])) as ArrayRef, true), - - + ( + "FixedSizeBinary", + Arc::new(FixedSizeBinaryArray::from(vec![&[1u8][..], &[2u8][..]])) as ArrayRef, + true, + ), + ( + "FixedSizeList", + Arc::new(FixedSizeListArray::from_iter_primitive::( + vec![ + Some(vec![Some(0), Some(1), Some(2)]), + None, + // Some(vec![Some(3), None, Some(5)]), + // Some(vec![Some(6), Some(7)]), + ], + 3, + )) as ArrayRef, + true, + ), + ( + "Float32", + Arc::new(Float32Array::from(vec![1.0, -1.0])) as ArrayRef, + true, + ), + ( + "Float64", + Arc::new(Float64Array::from(vec![1.0, -1.0])) as ArrayRef, + true, + ), ("Int8", Arc::new(Int8Array::from(vec![1i8, -2i8])) as ArrayRef, true), // ("Int8Dictionary", Arc::new(Int8DictionaryArray::from_iter([Some("a"), None])) as ArrayRef, true), ("Int16", Arc::new(Int16Array::from(vec![1i16, -2i16])) as ArrayRef, true), @@ -356,44 +375,56 @@ mod insert_tests { // ParquetError(NYI("Attempting to write an Arrow interval type MonthDayNano to parquet that is not yet implemented")) //("IntervalMonthDayNano", Arc::new(IntervalMonthDayNanoArray::from(vec![1])) as ArrayRef, true), // ("IntervalYearMonth", Arc::new(IntervalYearMonthArray::from(vec![1, 2])) as ArrayRef, true), - - ("Map", Arc::new({ - let string_builder = StringBuilder::new(); - let int_builder = Int32Builder::with_capacity(4); - - // Construct `[{"joe": 1}, {"blogs": 2, "foo": 4}]` - let mut builder = MapBuilder::new(None, string_builder, int_builder); - - builder.keys().append_value("joe"); - builder.values().append_value(1); - builder.append(true).unwrap(); - - builder.keys().append_value("blogs"); - builder.values().append_value(2); - builder.keys().append_value("foo"); - builder.values().append_value(4); - builder.append(true).unwrap(); - - builder.finish() - }) as ArrayRef, true), - + ( + "Map", + Arc::new({ + let string_builder = StringBuilder::new(); + let int_builder = Int32Builder::with_capacity(4); + + // Construct `[{"joe": 1}, {"blogs": 2, "foo": 4}]` + let mut builder = MapBuilder::new(None, string_builder, int_builder); + + builder.keys().append_value("joe"); + builder.values().append_value(1); + builder.append(true).unwrap(); + + builder.keys().append_value("blogs"); + builder.values().append_value(2); + builder.keys().append_value("foo"); + builder.values().append_value(4); + builder.append(true).unwrap(); + + builder.finish() + }) as ArrayRef, + true, + ), ("Null", Arc::new(NullArray::new(2)) as ArrayRef, true), - - ("LargeBinary", Arc::new(LargeBinaryArray::from_vec(vec![&[1u8], &[2u8, 3u8]])) as ArrayRef, true), - ("LargeString", Arc::new(LargeStringArray::from(vec!["1", ""])) as ArrayRef, true), - - ("List", Arc::new(ListArray::from_iter_primitive::(vec![ + ( + "LargeBinary", + Arc::new(LargeBinaryArray::from_vec(vec![&[1u8], &[2u8, 3u8]])) as ArrayRef, + true, + ), + ( + "LargeString", + Arc::new(LargeStringArray::from(vec!["1", ""])) as ArrayRef, + true, + ), + ( + "List", + Arc::new(ListArray::from_iter_primitive::(vec![ Some(vec![Some(0), Some(1), Some(2)]), None, // Some(vec![Some(3), None, Some(5)]), // Some(vec![Some(6), Some(7)]), - ])) as ArrayRef, true), - + ])) as ArrayRef, + true, + ), // ParquetError(ArrowError("Converting RunEndEncodedType to parquet not supported")) // ("Run", Arc::new(RunArray::::from_iter([Some("a"), None])) as ArrayRef, true), - ("String", Arc::new(StringArray::from(vec!["1", ""])) as ArrayRef, true), - ("Struct", Arc::new(StructArray::from(vec![ + ( + "Struct", + Arc::new(StructArray::from(vec![ ( Arc::new(Field::new("b", DataType::Boolean, false)), Arc::new(BooleanArray::from(vec![false, true])) as ArrayRef, @@ -402,32 +433,72 @@ mod insert_tests { Arc::new(Field::new("c", DataType::Int32, false)), Arc::new(Int32Array::from(vec![42, 31])) as ArrayRef, ), - ])) as ArrayRef, true), - - ("Time32Millisecond", Arc::new(Time32MillisecondArray::from(vec![1i32, -2i32])) as ArrayRef, true), - ("Time32Second", Arc::new(Time32SecondArray::from(vec![1i32, -2i32])) as ArrayRef, true), - ("Time64Microsecond", Arc::new(Time64MicrosecondArray::from(vec![1i64, -2i64])) as ArrayRef, true), - ("Time64Nanosecond", Arc::new(Time64NanosecondArray::from(vec![1i64, -2i64])) as ArrayRef, true), - ("TimestampMicrosecond", Arc::new(TimestampMicrosecondArray::from(vec![1i64, -2i64])) as ArrayRef, true), - ("TimestampMillisecond", Arc::new(TimestampMillisecondArray::from(vec![1i64, -2i64])) as ArrayRef, true), - ("TimestampNanosecond", Arc::new(TimestampNanosecondArray::from(vec![1i64, -2i64])) as ArrayRef, true), - ("TimestampSecond", Arc::new(TimestampSecondArray::from(vec![1i64, -2i64])) as ArrayRef, true), - + ])) as ArrayRef, + true, + ), + ( + "Time32Millisecond", + Arc::new(Time32MillisecondArray::from(vec![1i32, -2i32])) as ArrayRef, + true, + ), + ( + "Time32Second", + Arc::new(Time32SecondArray::from(vec![1i32, -2i32])) as ArrayRef, + true, + ), + ( + "Time64Microsecond", + Arc::new(Time64MicrosecondArray::from(vec![1i64, -2i64])) as ArrayRef, + true, + ), + ( + "Time64Nanosecond", + Arc::new(Time64NanosecondArray::from(vec![1i64, -2i64])) as ArrayRef, + true, + ), + ( + "TimestampMicrosecond", + Arc::new(TimestampMicrosecondArray::from(vec![1i64, -2i64])) as ArrayRef, + true, + ), + ( + "TimestampMillisecond", + Arc::new(TimestampMillisecondArray::from(vec![1i64, -2i64])) as ArrayRef, + true, + ), + ( + "TimestampNanosecond", + Arc::new(TimestampNanosecondArray::from(vec![1i64, -2i64])) as ArrayRef, + true, + ), + ( + "TimestampSecond", + Arc::new(TimestampSecondArray::from(vec![1i64, -2i64])) as ArrayRef, + true, + ), ("UInt8", Arc::new(UInt8Array::from(vec![1u8, 2u8])) as ArrayRef, true), // ("UInt8Dictionary", Arc::new(UInt8DictionaryArray::from_iter([Some("a"), None])) as ArrayRef, true), - ("UInt16", Arc::new(UInt16Array::from(vec![1u16, 2u16])) as ArrayRef, true), + ( + "UInt16", + Arc::new(UInt16Array::from(vec![1u16, 2u16])) as ArrayRef, + true, + ), // ("UInt16Dictionary", Arc::new(UInt16DictionaryArray::from_iter([Some("a"), None])) as ArrayRef, true), - ("UInt32", Arc::new(UInt32Array::from(vec![1u32, 2u32])) as ArrayRef, true), + ( + "UInt32", + Arc::new(UInt32Array::from(vec![1u32, 2u32])) as ArrayRef, + true, + ), // ("UInt32Dictionary", Arc::new(UInt32DictionaryArray::from_iter([Some("a"), None])) as ArrayRef, true), - ("UInt64", Arc::new(UInt64Array::from(vec![1u64, 2u64])) as ArrayRef, true), + ( + "UInt64", + Arc::new(UInt64Array::from(vec![1u64, 2u64])) as ArrayRef, + true, + ), // ("UInt64Dictionary", Arc::new(UInt64DictionaryArray::from_iter([Some("a"), None])) as ArrayRef, true), ]; let record_batch = RecordBatch::try_from_iter_with_nullable(iter).unwrap(); - init_table( - client.clone(), - record_batch.schema(), - table_name, - ).await?; + init_table(client.clone(), record_batch.schema(), table_name).await?; do_insert(record_batch, table_name).await?; check_insert(client.clone(), table_name, vec![], None, &[ "+---------+--------+------------+---------------------+---------------+--------------+-----------------+---------------+---------+---------+------+-------+-------+-------+--------------------+------+-------------+-------------+-----------+--------+-------------------+-----------------------------------------------------------------------------+------------------------------------------------------------------------+-----------------------------------------------------------------------------+----------------------------------------------------------------------------+----------------------------+-------------------------+-------------------------------+---------------------+-------+--------+--------+--------+", @@ -439,32 +510,28 @@ mod insert_tests { ]).await } - #[tokio::test] - async fn test_all_cases() -> Result<()> { - + async fn test_all_cases() -> Result<()> { test_insert_into_append().await?; test_insert_into_append_by_position().await?; test_insert_into_append_partitioned_table().await?; test_insert_into_append_non_partitioned_table_and_read_with_filter().await?; test_insert_into_append_partitioned_table_and_read_with_partition_filter().await?; - + test_insert_into_fails_when_missing_a_column().await?; test_insert_into_fails_when_an_extra_column_is_present_but_can_evolve_schema().await?; test_datatypes().await?; - + // overwrite case // todo: insert_overwrite is not supported by datafusion 27.0 // test_insert_into_overwrite_non_partitioned_table().await?; // test_insert_into_overwrite_by_position().await?; - // todo: + // todo: // test_insert_info_schema_enforcement().await?; // test_insert_info_struct_types_and_schema_enforcement().await?; Ok(()) - } - - + } } diff --git a/rust/lakesoul-datafusion/src/test/mod.rs b/rust/lakesoul-datafusion/src/test/mod.rs index 2ffb2c6fe..5ec114b77 100644 --- a/rust/lakesoul-datafusion/src/test/mod.rs +++ b/rust/lakesoul-datafusion/src/test/mod.rs @@ -12,7 +12,7 @@ mod upsert_tests; // mod streaming_tests; #[ctor::ctor] -fn init() { +fn init() { lakesoul_io::tokio::runtime::Builder::new_multi_thread() .enable_all() .build() @@ -22,4 +22,4 @@ fn init() { client.meta_cleanup().await.unwrap(); println!("clean metadata"); }) -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/test/upsert_tests.rs b/rust/lakesoul-datafusion/src/test/upsert_tests.rs index 0517110ac..e7c0c6baf 100644 --- a/rust/lakesoul-datafusion/src/test/upsert_tests.rs +++ b/rust/lakesoul-datafusion/src/test/upsert_tests.rs @@ -2,20 +2,20 @@ // // SPDX-License-Identifier: Apache-2.0 mod upsert_with_io_config_tests { - use std::sync::Arc; + use chrono::naive::NaiveDate; use std::env; use std::path::PathBuf; + use std::sync::Arc; use std::time::SystemTime; - use chrono::naive::NaiveDate; + use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampMicrosecondArray}; + use arrow::datatypes::{Field, Schema, SchemaRef, TimeUnit}; use arrow::record_batch::RecordBatch; use datafusion::assert_batches_eq; - use lakesoul_io::lakesoul_reader::{LakeSoulReader, SyncSendableMutableLakeSoulReader}; - use tokio::runtime::Builder; - use arrow::array::{ArrayRef, Int32Array,TimestampMicrosecondArray, StringArray}; - use arrow::datatypes::{Schema, SchemaRef, Field, TimeUnit}; use lakesoul_io::lakesoul_io_config::LakeSoulIOConfigBuilder; + use lakesoul_io::lakesoul_reader::{LakeSoulReader, SyncSendableMutableLakeSoulReader}; use lakesoul_io::lakesoul_writer::SyncSendableMutableLakeSoulWriter; + use tokio::runtime::Builder; use arrow::array::Int64Array; @@ -24,20 +24,28 @@ mod upsert_with_io_config_tests { v2(i32), } - fn init_table(batch: RecordBatch, table_name: &str, pks:Vec) -> LakeSoulIOConfigBuilder { - + fn init_table(batch: RecordBatch, table_name: &str, pks: Vec) -> LakeSoulIOConfigBuilder { let builder = LakeSoulIOConfigBuilder::new() - .with_schema(batch.schema()) - .with_primary_keys(pks); + .with_schema(batch.schema()) + .with_primary_keys(pks); execute_upsert(batch, table_name, builder.clone()) } - fn check_upsert(batch: RecordBatch, table_name: &str, selected_cols: Vec<&str>, filters: Option, builder: LakeSoulIOConfigBuilder, expected: &[&str]) -> LakeSoulIOConfigBuilder { + fn check_upsert( + batch: RecordBatch, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + builder: LakeSoulIOConfigBuilder, + expected: &[&str], + ) -> LakeSoulIOConfigBuilder { let builder = execute_upsert(batch, table_name, builder.clone()); - let builder = builder - .with_schema(SchemaRef::new(Schema::new( - selected_cols.iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() - ))); + let builder = builder.with_schema(SchemaRef::new(Schema::new( + selected_cols + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), + ))); let builder = if let Some(filters) = filters { builder.with_filter_str(filters) } else { @@ -45,19 +53,44 @@ mod upsert_with_io_config_tests { }; let config = builder.clone().build(); - let mut reader = SyncSendableMutableLakeSoulReader::new(LakeSoulReader::new(config).unwrap(), Builder::new_current_thread().build().unwrap()); + let mut reader = SyncSendableMutableLakeSoulReader::new( + LakeSoulReader::new(config).unwrap(), + Builder::new_current_thread().build().unwrap(), + ); let _ = reader.start_blocked(); let result = reader.next_rb_blocked(); assert_batches_eq!(expected, &[result.unwrap().unwrap()]); builder } - fn execute_upsert(batch: RecordBatch, table_name: &str, builder: LakeSoulIOConfigBuilder) -> LakeSoulIOConfigBuilder { - let file = [env::temp_dir().to_str().unwrap(), table_name, format!("{}.parquet", SystemTime::now().duration_since(SystemTime::UNIX_EPOCH).unwrap().as_millis().to_string()).as_str()].iter().collect::().to_str().unwrap().to_string(); + fn execute_upsert( + batch: RecordBatch, + table_name: &str, + builder: LakeSoulIOConfigBuilder, + ) -> LakeSoulIOConfigBuilder { + let file = [ + env::temp_dir().to_str().unwrap(), + table_name, + format!( + "{}.parquet", + SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_millis() + .to_string() + ) + .as_str(), + ] + .iter() + .collect::() + .to_str() + .unwrap() + .to_string(); let builder = builder.with_file(file.clone()).with_schema(batch.schema()); let config = builder.clone().build(); - let writer = SyncSendableMutableLakeSoulWriter::try_new(config, Builder::new_current_thread().build().unwrap()).unwrap(); + let writer = + SyncSendableMutableLakeSoulWriter::try_new(config, Builder::new_current_thread().build().unwrap()).unwrap(); let _ = writer.write_batch(batch); let _ = writer.flush_and_close(); builder @@ -68,7 +101,11 @@ mod upsert_with_io_config_tests { .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } @@ -77,71 +114,103 @@ mod upsert_with_io_config_tests { .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } fn create_batch_string(names: Vec<&str>, values: Vec<&[&str]>) -> RecordBatch { - let values=values + let values = values .into_iter() .map(|vec| Arc::new(StringArray::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } - fn create_batch_i32_and_timestamp(names: Vec<&str>, values: Vec<&[i32]>, timestamp:Vec) -> RecordBatch { + fn create_batch_i32_and_timestamp(names: Vec<&str>, values: Vec<&[i32]>, timestamp: Vec) -> RecordBatch { let mut values = values .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); let timestamp = Arc::new(TimestampMicrosecondArray::from(timestamp)) as ArrayRef; values.push(timestamp); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } fn create_batch_str_or_i32(names: Vec<&str>, values: Vec<&[str_or_i32]>) -> RecordBatch { let values = values .into_iter() - .map(|vec| - match vec[0] { - str_or_i32::v1(_) => { - let vec=vec.into_iter().map(|val| - match val { - str_or_i32::v1(v1) => Some(*v1), - str_or_i32::v2(v2) => None, - } - ).map(|val| val.unwrap()).collect::>(); - Arc::new(StringArray::from(vec)) as ArrayRef - } - str_or_i32::v2(_) => { - let vec=vec.into_iter().map(|val| - match val { - str_or_i32::v1(v1) => None, - str_or_i32::v2(v2) => Some(v2), - } - ).map(|val| *val.unwrap()).collect::>(); - Arc::new(Int32Array::from(vec)) as ArrayRef - } + .map(|vec| match vec[0] { + str_or_i32::v1(_) => { + let vec = vec + .into_iter() + .map(|val| match val { + str_or_i32::v1(v1) => Some(*v1), + str_or_i32::v2(v2) => None, + }) + .map(|val| val.unwrap()) + .collect::>(); + Arc::new(StringArray::from(vec)) as ArrayRef } - ).collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + str_or_i32::v2(_) => { + let vec = vec + .into_iter() + .map(|val| match val { + str_or_i32::v1(v1) => None, + str_or_i32::v2(v2) => Some(v2), + }) + .map(|val| *val.unwrap()) + .collect::>(); + Arc::new(Int32Array::from(vec)) as ArrayRef + } + }) + .collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } - fn check_upsert_i32_and_timestamp(batch: RecordBatch, table_name: &str, selected_cols: Vec<&str>, filters: Option, builder: LakeSoulIOConfigBuilder, expected: &[&str]) -> LakeSoulIOConfigBuilder { + fn check_upsert_i32_and_timestamp( + batch: RecordBatch, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + builder: LakeSoulIOConfigBuilder, + expected: &[&str], + ) -> LakeSoulIOConfigBuilder { let builder = execute_upsert(batch, table_name, builder.clone()); - let builder = builder - .with_schema(SchemaRef::new(Schema::new( - selected_cols.iter().map(|col| - if *col=="timestamp"{ - Field::new(*col, arrow::datatypes::DataType::Timestamp(TimeUnit::Microsecond, None), true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - } - ).collect::>() - ))); + let builder = builder.with_schema(SchemaRef::new(Schema::new( + selected_cols + .iter() + .map(|col| { + if *col == "timestamp" { + Field::new( + *col, + arrow::datatypes::DataType::Timestamp(TimeUnit::Microsecond, None), + true, + ) + } else { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } + }) + .collect::>(), + ))); let builder = if let Some(filters) = filters { builder.with_filter_str(filters) } else { @@ -149,19 +218,31 @@ mod upsert_with_io_config_tests { }; let config = builder.clone().build(); - let mut reader = SyncSendableMutableLakeSoulReader::new(LakeSoulReader::new(config).unwrap(), Builder::new_current_thread().build().unwrap()); + let mut reader = SyncSendableMutableLakeSoulReader::new( + LakeSoulReader::new(config).unwrap(), + Builder::new_current_thread().build().unwrap(), + ); let _ = reader.start_blocked(); let result = reader.next_rb_blocked(); assert_batches_eq!(expected, &[result.unwrap().unwrap()]); builder } - fn check_upsert_string(batch: RecordBatch, table_name: &str, selected_cols: Vec<&str>, filters: Option, builder: LakeSoulIOConfigBuilder, expected: &[&str]) -> LakeSoulIOConfigBuilder { + fn check_upsert_string( + batch: RecordBatch, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + builder: LakeSoulIOConfigBuilder, + expected: &[&str], + ) -> LakeSoulIOConfigBuilder { let builder = execute_upsert(batch, table_name, builder.clone()); - let builder = builder - .with_schema(SchemaRef::new(Schema::new( - selected_cols.iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Utf8, true)).collect::>() - ))); + let builder = builder.with_schema(SchemaRef::new(Schema::new( + selected_cols + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Utf8, true)) + .collect::>(), + ))); let builder = if let Some(filters) = filters { builder.with_filter_str(filters) } else { @@ -169,25 +250,37 @@ mod upsert_with_io_config_tests { }; let config = builder.clone().build(); - let mut reader = SyncSendableMutableLakeSoulReader::new(LakeSoulReader::new(config).unwrap(), Builder::new_current_thread().build().unwrap()); + let mut reader = SyncSendableMutableLakeSoulReader::new( + LakeSoulReader::new(config).unwrap(), + Builder::new_current_thread().build().unwrap(), + ); let _ = reader.start_blocked(); let result = reader.next_rb_blocked(); assert_batches_eq!(expected, &[result.unwrap().unwrap()]); builder } - fn check_upsert_string_or_i32(batch: RecordBatch, table_name: &str, selected_cols: Vec<&str>, filters: Option, builder: LakeSoulIOConfigBuilder, expected: &[&str]) -> LakeSoulIOConfigBuilder { + fn check_upsert_string_or_i32( + batch: RecordBatch, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + builder: LakeSoulIOConfigBuilder, + expected: &[&str], + ) -> LakeSoulIOConfigBuilder { let builder = execute_upsert(batch, table_name, builder.clone()); - let builder = builder - .with_schema(SchemaRef::new(Schema::new( - selected_cols.iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() - ))); + let builder = builder.with_schema(SchemaRef::new(Schema::new( + selected_cols + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), + ))); let builder = if let Some(filters) = filters { builder.with_filter_str(filters) } else { @@ -195,7 +288,10 @@ mod upsert_with_io_config_tests { }; let config = builder.clone().build(); - let mut reader = SyncSendableMutableLakeSoulReader::new(LakeSoulReader::new(config).unwrap(), Builder::new_current_thread().build().unwrap()); + let mut reader = SyncSendableMutableLakeSoulReader::new( + LakeSoulReader::new(config).unwrap(), + Builder::new_current_thread().build().unwrap(), + ); let _ = reader.start_blocked(); let result = reader.next_rb_blocked(); assert_batches_eq!(expected, &[result.unwrap().unwrap()]); @@ -206,16 +302,23 @@ mod upsert_with_io_config_tests { fn test_merge_same_column_i32() { let table_name = "merge-same_column"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["range", "hash", "value"], + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + vec!["range", "hash", "value"], None, - builder.clone(), + builder.clone(), &[ "+----------+------+-------+", "| range | hash | value |", @@ -226,7 +329,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | 44 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] + ], ); } @@ -234,16 +337,23 @@ mod upsert_with_io_config_tests { fn test_merge_different_column_i32() { let table_name = "merge-different_column"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "name"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["range", "hash", "value", "name"], - None, - builder.clone(), + create_batch_i32( + vec!["range", "hash", "name"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + vec!["range", "hash", "value", "name"], + None, + builder.clone(), &[ "+----------+------+-------+------+", "| range | hash | value | name |", @@ -254,7 +364,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | | 44 |", "| 20201102 | 4 | 4 | |", "+----------+------+-------+------+", - ] + ], ); } @@ -262,17 +372,24 @@ mod upsert_with_io_config_tests { fn test_merge_different_columns_and_filter_by_non_selected_columns_i32() { let table_name = "merge-different_columns_and_filter_by_non_selected_columns_i32"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "name"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["range", "hash", "value"], + create_batch_i32( + vec!["range", "hash", "name"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + vec!["range", "hash", "value"], Some("and(noteq(name, null), gt(name, 0))".to_string()), - builder.clone(), - &[ + builder.clone(), + &[ "+----------+------+-------+", "| range | hash | value |", "+----------+------+-------+", @@ -280,7 +397,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 3 | 3 |", "| 20201101 | 4 | |", "+----------+------+-------+", - ] + ], ); } @@ -288,17 +405,29 @@ mod upsert_with_io_config_tests { fn test_merge_different_columns_and_filter_partial_rows_i32() { let table_name = "merge-different_columns_and_filter_partial_rows_i32"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4], &[11, 22, 33, 44]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201101, 20201101, 20201101, 20201102], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[11, 22, 33, 44], + ], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 3, 4, 4], &[2, 4, 5, 5]]), - table_name, - vec!["range", "hash", "value", "name"], + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 3, 4, 4], &[2, 4, 5, 5]], + ), + table_name, + vec!["range", "hash", "value", "name"], Some("and(and(noteq(value, null), lt(value, 5)),and(noteq(name, null), gt(name, 0)))".to_string()), - builder.clone(), - &[ + builder.clone(), + &[ "+----------+------+-------+------+", "| range | hash | value | name |", "+----------+------+-------+------+", @@ -306,7 +435,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 2 | 2 | 22 |", "| 20201101 | 3 | 4 | 33 |", "+----------+------+-------+------+", - ] + ], ); } @@ -314,18 +443,25 @@ mod upsert_with_io_config_tests { fn test_merge_one_file_with_empty_batch_i32() { let table_name = "merge_one_file_with_empty_batch"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() - ))), - table_name, - vec!["range", "hash", "value"], + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), + ))), + table_name, + vec!["range", "hash", "value"], Some("and(noteq(value, null), lt(value, 3))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+------+-------+", "| range | hash | value |", @@ -333,7 +469,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 1 | 1 |", "| 20201101 | 2 | 2 |", "+----------+------+-------+", - ] + ], ); } @@ -341,23 +477,35 @@ mod upsert_with_io_config_tests { fn test_merge_multi_files_with_empty_batch_i32() { let table_name = "merge_multi_files_with_empty_batch"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102, 20201102], &[1, 2, 3, 4, 1], &[1, 2, 3, 4, 1]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![ + &[20201101, 20201101, 20201101, 20201102, 20201102], + &[1, 2, 3, 4, 1], + &[1, 2, 3, 4, 1], + ], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + let builder = execute_upsert( create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102], &[4], &[5]]), - table_name, - builder); - + table_name, + builder, + ); + check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() - ))), - table_name, - vec!["range", "hash", "value"], + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), + ))), + table_name, + vec!["range", "hash", "value"], Some("and(noteq(value, null), lt(value, 3))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+------+-------+", "| range | hash | value |", @@ -366,7 +514,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 2 | 2 |", "| 20201102 | 1 | 1 |", "+----------+------+-------+", - ] + ], ); } @@ -375,13 +523,20 @@ mod upsert_with_io_config_tests { // require metadata checker let table_name = "test_basic_upsert_same_columns"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), table_name, - vec!["range".to_string(), "hash".to_string()]); + vec!["range".to_string(), "hash".to_string()], + ); check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), ))), table_name, vec!["range", "hash", "value"], @@ -396,11 +551,14 @@ mod upsert_with_io_config_tests { "| 20201101 | 3 | 3 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] + ], ); check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value"], None, @@ -415,7 +573,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | 44 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] + ], ); } @@ -424,12 +582,19 @@ mod upsert_with_io_config_tests { // require metadata checker let table_name = "basic_upsert_different_columns"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), table_name, - vec!["range".to_string(), "hash".to_string()]); + vec!["range".to_string(), "hash".to_string()], + ); check_upsert( - create_batch_i32(vec!["range", "hash", "name"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "name"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value", "name"], None, @@ -444,7 +609,7 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | | 44 |", "| 20201102 | 4 | 4 | |", "+----------+------+-------+------+", - ] + ], ); } @@ -467,16 +632,23 @@ mod upsert_with_io_config_tests { fn test_upsert_without_range_parqitions_i32() { let table_name = "upsert_without_range_parqitions"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + vec!["hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["range", "hash", "value"], + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + vec!["range", "hash", "value"], None, - builder.clone(), + builder.clone(), &[ "+----------+------+-------+", "| range | hash | value |", @@ -486,9 +658,8 @@ mod upsert_with_io_config_tests { "| 20201101 | 3 | 33 |", "| 20201101 | 4 | 44 |", "+----------+------+-------+", - ] + ], ); - } #[test] @@ -500,16 +671,40 @@ mod upsert_with_io_config_tests { fn test_upsert_with_multiple_range_and_hash_parqitions_i32() { let table_name = "upsert_with_multiple_range_and_hash_parqitions"; let builder = init_table( - create_batch_i32(vec!["range1", "range2", "hash1", "hash2", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range1".to_string(), "range2".to_string(), "hash1".to_string(), "hash2".to_string()]); - + create_batch_i32( + vec!["range1", "range2", "hash1", "hash2", "value"], + vec![ + &[20201101, 20201101, 20201101, 20201102], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + vec![ + "range1".to_string(), + "range2".to_string(), + "hash1".to_string(), + "hash2".to_string(), + ], + ); + check_upsert( - create_batch_i32(vec!["range1", "range2", "hash1", "hash2", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[1, 3, 4],&[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["range1", "range2", "hash1", "hash2", "value"], + create_batch_i32( + vec!["range1", "range2", "hash1", "hash2", "value"], + vec![ + &[20201101, 20201101, 20201101], + &[1, 3, 4], + &[1, 3, 4], + &[1, 3, 4], + &[11, 33, 44], + ], + ), + table_name, + vec!["range1", "range2", "hash1", "hash2", "value"], None, - builder.clone(), + builder.clone(), &[ "+----------+--------+-------+-------+-------+", "| range1 | range2 | hash1 | hash2 | value |", @@ -520,15 +715,13 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | 4 | 4 | 44 |", "| 20201102 | 4 | 4 | 4 | 4 |", "+----------+--------+-------+-------+-------+", - ] + ], ); - } #[test] - fn test_source_dataframe_without_partition_columns(){ + fn test_source_dataframe_without_partition_columns() { // require metadata checker - } #[test] @@ -540,16 +733,29 @@ mod upsert_with_io_config_tests { fn test_filter_requested_columns_upsert_1_times_i32() { let table_name = "filter_requested_columns_upsert_1_times"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101, 20201101, 20201101], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![ + &[20201101, 20201101, 20201101, 20201101], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["range", "hash", "value", "name", "age"], + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + vec!["range", "hash", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+------+-------+------+-----+", "| range | hash | value | name | age |", @@ -558,30 +764,51 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 33 | | |", "| 20201102 | 4 | 44 | | |", "+----------+------+-------+------+-----+", - ] + ], ); - } #[test] fn test_filter_requested_columns_upsert_2_times_i32() { let table_name = "filter_requested_columns_upsert_2_times"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101, 20201101, 20201101], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![ + &[20201101, 20201101, 20201101, 20201101], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + let builder = execute_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), - table_name, - builder); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + builder, + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201102, 20201102, 20201102], &[1, 2, 3], &[111, 222, 333], &[11, 22, 33]]), - table_name, - vec!["range", "hash", "value", "name", "age"], + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201102, 20201102, 20201102], + &[1, 2, 3], + &[111, 222, 333], + &[11, 22, 33], + ], + ), + table_name, + vec!["range", "hash", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+------+-------+------+-----+", "| range | hash | value | name | age |", @@ -591,7 +818,7 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 333 | 33 | |", "| 20201102 | 4 | 44 | | |", "+----------+------+-------+------+-----+", - ] + ], ); } @@ -599,44 +826,69 @@ mod upsert_with_io_config_tests { fn test_filter_requested_columns_upsert_3_times_i32() { let table_name = "filter_requested_columns_upsert_3_times"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101, 20201101, 20201101], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![ + &[20201101, 20201101, 20201101, 20201101], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + let builder = execute_upsert( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201102, 20201102, 20201102], &[1, 2, 3], &[111, 222, 333], &[11, 22, 33]]), - table_name, - builder); - + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201102, 20201102, 20201102], + &[1, 2, 3], + &[111, 222, 333], + &[11, 22, 33], + ], + ), + table_name, + builder, + ); + let builder = execute_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), - table_name, - builder); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + builder, + ); + /*** - !!! Error all below conditions are satisfied - 1. entire 'value' column is null - 2. filter exists - 3. filter pushed down into TableProvider with TableProviderFilterPushDown::Inexact - 4. SessionConfig.execution.parquet.pruning = true (equivalent SessionConfig.with_parquet_pruning(true)) - 5. SessionConfig.execution.parquet.enable_page_index = true - 6. - */ + !!! Error all below conditions are satisfied + 1. entire 'value' column is null + 2. filter exists + 3. filter pushed down into TableProvider with TableProviderFilterPushDown::Inexact + 4. SessionConfig.execution.parquet.pruning = true (equivalent SessionConfig.with_parquet_pruning(true)) + 5. SessionConfig.execution.parquet.enable_page_index = true + 6. + */ check_upsert( create_batch_optional_i32( - vec!["range", "hash", "age", "name", "value"], + vec!["range", "hash", "age", "name", "value"], vec![ - &[Some(20201102), Some(20201102)], - &[Some(1), Some(3)], - &[Some(111), Some(333)], - &[Some(11), Some(33)], - &[None, Some(3333)]]), - // &[None, None]]), - table_name, - vec!["range", "hash", "value", "name", "age"], + &[Some(20201102), Some(20201102)], + &[Some(1), Some(3)], + &[Some(111), Some(333)], + &[Some(11), Some(33)], + &[None, Some(3333)], + ], + ), + // &[None, None]]), + table_name, + vec!["range", "hash", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), // None, - builder.clone(), + builder.clone(), &[ "+----------+------+-------+------+-----+", "| range | hash | value | name | age |", @@ -647,36 +899,34 @@ mod upsert_with_io_config_tests { // "| 20201102 | 3 | | 33 | 333 |", "| 20201102 | 4 | 44 | | |", "+----------+------+-------+------+-----+", - ] + ], ); - } #[test] fn test_select_requested_columns_without_hash_columns_upsert_1_times_i32() { let table_name = "select_requested_columns_without_hash_columns_upsert_1_times"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); - + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), - table_name, - vec!["age"], + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + vec!["age"], None, - builder.clone(), + builder.clone(), &[ - "+-----+", - "| age |", - "+-----+", - "| 1 |", - "| 2 |", - "| |", - "| |", - "| |", - "+-----+", - ] + "+-----+", "| age |", "+-----+", "| 1 |", "| 2 |", "| |", "| |", "| |", "+-----+", + ], ); } @@ -684,49 +934,64 @@ mod upsert_with_io_config_tests { fn test_select_requested_columns_without_hash_columns_upsert_2_times_i32() { let table_name = "select_requested_columns_without_hash_columns_upsert_2_times"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); let builder = execute_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), - table_name, - builder); - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, + builder, + ); + check_upsert( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201102, 20201102, 20201102], &[1, 2, 3], &[111, 222, 333], &[11, 22, 33]]), - table_name, - vec!["age"], + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201102, 20201102, 20201102], + &[1, 2, 3], + &[111, 222, 333], + &[11, 22, 33], + ], + ), + table_name, + vec!["age"], None, - builder.clone(), + builder.clone(), &[ + "+-----+", "| age |", "+-----+", "| 1 |", "| 2 |", "| |", "| |", "| |", "| |", "+-----+", - "| age |", - "+-----+", - "| 1 |", - "| 2 |", - "| |", - "| |", - "| |", - "| |", - "+-----+", - ] + ], ); } #[test] - fn test_derange_hash_key_and_data_schema_order_int_type_upsert_1_times_i32(){ + fn test_derange_hash_key_and_data_schema_order_int_type_upsert_1_times_i32() { let table_name = "derange_hash_key_and_data_schema_order_int_type_upsert_1_times_i32"; let builder = init_table( - create_batch_i32(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2],&[1, 2]]), - table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + create_batch_i32( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); check_upsert( - create_batch_i32(vec!["range", "hash1", "hash2", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]]), - table_name, + create_batch_i32( + vec!["range", "hash1", "hash2", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]], + ), + table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+-------+-------+-------+------+-----+", "| range | hash1 | hash2 | value | name | age |", @@ -735,27 +1000,43 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 32 | 3 | | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] + ], ); } #[test] - fn test_derange_hash_key_and_data_schema_order_int_type_upsert_2_times_i32(){ + fn test_derange_hash_key_and_data_schema_order_int_type_upsert_2_times_i32() { let table_name = "derange_hash_key_and_data_schema_order_int_type_upsert_2_times_i32"; let builder = init_table( - create_batch_i32(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2],&[1, 2]]), - table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + create_batch_i32( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); let builder = execute_upsert( - create_batch_i32(vec!["range", "hash1", "hash2", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]]), - table_name, - builder); + create_batch_i32( + vec!["range", "hash1", "hash2", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]], + ), + table_name, + builder, + ); check_upsert( - create_batch_i32(vec!["range", "hash2", "name", "hash1"], vec![&[20201102, 20201102, 20201102], &[12, 22, 32], &[11, 22, 33], &[1, 2, 3]]), - table_name, + create_batch_i32( + vec!["range", "hash2", "name", "hash1"], + vec![ + &[20201102, 20201102, 20201102], + &[12, 22, 32], + &[11, 22, 33], + &[1, 2, 3], + ], + ), + table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+-------+-------+-------+------+-----+", "| range | hash1 | hash2 | value | name | age |", @@ -765,31 +1046,57 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 32 | 3 | 33 | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] + ], ); } #[test] - fn test_derange_hash_key_and_data_schema_order_int_type_upsert_3_times_i32(){ + fn test_derange_hash_key_and_data_schema_order_int_type_upsert_3_times_i32() { let table_name = "derange_hash_key_and_data_schema_order_int_type_upsert_3_times_i32"; let builder = init_table( - create_batch_i32(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2],&[1, 2]]), - table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + create_batch_i32( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); let builder = execute_upsert( - create_batch_i32(vec!["range", "hash1", "hash2", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]]), - table_name, - builder); + create_batch_i32( + vec!["range", "hash1", "hash2", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]], + ), + table_name, + builder, + ); let builder = execute_upsert( - create_batch_i32(vec!["range", "hash2", "name", "hash1"], vec![&[20201102, 20201102, 20201102], &[12, 22, 32], &[11, 22, 33], &[1, 2, 3]]), - table_name, - builder); + create_batch_i32( + vec!["range", "hash2", "name", "hash1"], + vec![ + &[20201102, 20201102, 20201102], + &[12, 22, 32], + &[11, 22, 33], + &[1, 2, 3], + ], + ), + table_name, + builder, + ); check_upsert( - create_batch_i32(vec!["range", "age", "hash2", "name", "hash1"], vec![&[20201102, 20201102, 20201102], &[4567, 2345, 3456], &[42, 22, 32], &[456, 234, 345], &[4, 2, 3]]), - table_name, + create_batch_i32( + vec!["range", "age", "hash2", "name", "hash1"], + vec![ + &[20201102, 20201102, 20201102], + &[4567, 2345, 3456], + &[42, 22, 32], + &[456, 234, 345], + &[4, 2, 3], + ], + ), + table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(and(noteq(range, null), eq(range, 20201102)), noteq(value, null))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+-------+-------+-------+------+------+", "| range | hash1 | hash2 | value | name | age |", @@ -798,23 +1105,42 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 32 | 3 | 345 | 3456 |", "| 20201102 | 4 | 42 | 4 | 456 | 4567 |", "+----------+-------+-------+-------+------+------+", - ] + ], ); } #[test] - fn test_derange_hash_key_and_data_schema_order_string_type_upsert_1_times_i32(){ + fn test_derange_hash_key_and_data_schema_order_string_type_upsert_1_times_i32() { let table_name = "derange_hash_key_and_data_schema_order_string_type_upsert_1_times_i32"; let builder = init_table( - create_batch_string(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&["20201101", "20201101"], &["1", "2"], &["1", "2"], &["1", "2"], &["1", "2"],&["1", "2"]]), - table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + create_batch_string( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![ + &["20201101", "20201101"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + ], + ), + table_name, + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); check_upsert_string( - create_batch_string(vec!["range", "hash1", "hash2", "value"], vec![&["20201102", "20201102", "20201102"], &["1", "3", "4"], &["12", "32", "42"], &["1", "3", "4"]]), - table_name, + create_batch_string( + vec!["range", "hash1", "hash2", "value"], + vec![ + &["20201102", "20201102", "20201102"], + &["1", "3", "4"], + &["12", "32", "42"], + &["1", "3", "4"], + ], + ), + table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, ________20201102__))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+-------+-------+-------+------+-----+", "| range | hash1 | hash2 | value | name | age |", @@ -823,27 +1149,55 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 32 | 3 | | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] + ], ); } #[test] - fn test_derange_hash_key_and_data_schema_order_string_type_upsert_2_times_i32(){ + fn test_derange_hash_key_and_data_schema_order_string_type_upsert_2_times_i32() { let table_name = "derange_hash_key_and_data_schema_order_string_type_upsert_2_times_i32"; let builder = init_table( - create_batch_string(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&["20201101", "20201101"], &["1", "2"], &["1", "2"], &["1", "2"], &["1", "2"],&["1", "2"]]), - table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + create_batch_string( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![ + &["20201101", "20201101"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + ], + ), + table_name, + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); let builder = execute_upsert( - create_batch_string(vec!["range", "hash1", "hash2", "value"], vec![&["20201102", "20201102", "20201102"], &["1", "3", "4"], &["12", "32", "42"], &["1", "3", "4"]]), - table_name, - builder); + create_batch_string( + vec!["range", "hash1", "hash2", "value"], + vec![ + &["20201102", "20201102", "20201102"], + &["1", "3", "4"], + &["12", "32", "42"], + &["1", "3", "4"], + ], + ), + table_name, + builder, + ); check_upsert_string( - create_batch_string(vec!["range", "hash2", "name", "hash1"], vec![&["20201102", "20201102", "20201102"], &["12", "22", "32"], &["11", "22", "33"], &["1", "2", "3"]]), - table_name, + create_batch_string( + vec!["range", "hash2", "name", "hash1"], + vec![ + &["20201102", "20201102", "20201102"], + &["12", "22", "32"], + &["11", "22", "33"], + &["1", "2", "3"], + ], + ), + table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, ________20201102__))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+-------+-------+-------+------+-----+", "| range | hash1 | hash2 | value | name | age |", @@ -853,31 +1207,69 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 32 | 3 | 33 | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] + ], ); } #[test] - fn test_derange_hash_key_and_data_schema_order_string_type_upsert_3_times_i32(){ + fn test_derange_hash_key_and_data_schema_order_string_type_upsert_3_times_i32() { let table_name = "derange_hash_key_and_data_schema_order_string_type_upsert_3_times_i32"; let builder = init_table( - create_batch_string(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&["20201101", "20201101"], &["1", "2"], &["1", "2"], &["1", "2"], &["1", "2"],&["1", "2"]]), - table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + create_batch_string( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![ + &["20201101", "20201101"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + ], + ), + table_name, + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); let builder = execute_upsert( - create_batch_string(vec!["range", "hash1", "hash2", "value"], vec![&["20201102", "20201102", "20201102"], &["1", "3", "4"], &["12", "32", "42"], &["1", "3", "4"]]), - table_name, - builder); + create_batch_string( + vec!["range", "hash1", "hash2", "value"], + vec![ + &["20201102", "20201102", "20201102"], + &["1", "3", "4"], + &["12", "32", "42"], + &["1", "3", "4"], + ], + ), + table_name, + builder, + ); let builder = execute_upsert( - create_batch_string(vec!["range", "hash2", "name", "hash1"], vec![&["20201102", "20201102", "20201102"], &["12", "22", "32"], &["11", "22", "33"], &["1", "2", "3"]]), - table_name, - builder); + create_batch_string( + vec!["range", "hash2", "name", "hash1"], + vec![ + &["20201102", "20201102", "20201102"], + &["12", "22", "32"], + &["11", "22", "33"], + &["1", "2", "3"], + ], + ), + table_name, + builder, + ); check_upsert_string( - create_batch_string(vec!["range", "age", "hash2", "name", "hash1"], vec![&["20201102", "20201102", "20201102"], &["4567", "2345", "3456"], &["42", "22", "32"], &["456", "234", "345"], &["4", "2", "3"]]), - table_name, + create_batch_string( + vec!["range", "age", "hash2", "name", "hash1"], + vec![ + &["20201102", "20201102", "20201102"], + &["4567", "2345", "3456"], + &["42", "22", "32"], + &["456", "234", "345"], + &["4", "2", "3"], + ], + ), + table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(and(noteq(range, null), eq(range, ________20201102__)), noteq(value, null))".to_string()), - builder.clone(), + builder.clone(), &[ "+----------+-------+-------+-------+------+------+", "| range | hash1 | hash2 | value | name | age |", @@ -886,54 +1278,81 @@ mod upsert_with_io_config_tests { "| 20201102 | 3 | 32 | 3 | 345 | 3456 |", "| 20201102 | 4 | 42 | 4 | 456 | 4567 |", "+----------+-------+-------+-------+------+------+", - ] + ], ); } - #[test] - fn test_create_table_with_hash_key_disordered(){ + fn test_create_table_with_hash_key_disordered() { let table_name = "test_create_table_with_hash_key_disordered"; - let batch1=create_batch_str_or_i32(vec!["range", "v1", "hash1", "v2", "hash2"], - vec![&[str_or_i32::v1("range"), str_or_i32::v1("range"), str_or_i32::v1("range")], &[str_or_i32::v1("a1"), str_or_i32::v1("b1"), str_or_i32::v1("c1")], - &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], &[str_or_i32::v1("a2"), str_or_i32::v1("b2"), str_or_i32::v1("c2")], &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")]] + let batch1 = create_batch_str_or_i32( + vec!["range", "v1", "hash1", "v2", "hash2"], + vec![ + &[ + str_or_i32::v1("range"), + str_or_i32::v1("range"), + str_or_i32::v1("range"), + ], + &[str_or_i32::v1("a1"), str_or_i32::v1("b1"), str_or_i32::v1("c1")], + &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], + &[str_or_i32::v1("a2"), str_or_i32::v1("b2"), str_or_i32::v1("c2")], + &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + ], ); - let batch2=create_batch_str_or_i32(vec!["range", "hash1", "v1", "v2", "hash2"], - vec![&[str_or_i32::v1("range"), str_or_i32::v1("range"), str_or_i32::v1("range")], &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], &[str_or_i32::v1("a11"), str_or_i32::v1("b11"), str_or_i32::v1("c11")], - &[str_or_i32::v1("a22"), str_or_i32::v1("b22"), str_or_i32::v1("c22")], &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")]] + let batch2 = create_batch_str_or_i32( + vec!["range", "hash1", "v1", "v2", "hash2"], + vec![ + &[ + str_or_i32::v1("range"), + str_or_i32::v1("range"), + str_or_i32::v1("range"), + ], + &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], + &[str_or_i32::v1("a11"), str_or_i32::v1("b11"), str_or_i32::v1("c11")], + &[str_or_i32::v1("a22"), str_or_i32::v1("b22"), str_or_i32::v1("c22")], + &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + ], ); - let batch3=create_batch_str_or_i32(vec!["range", "v1", "hash1", "v2", "hash2"], - vec![&[str_or_i32::v1("range"), str_or_i32::v1("range"), str_or_i32::v1("range")], &[str_or_i32::v1("d1"), str_or_i32::v1("b111"), str_or_i32::v1("c111")], - &[str_or_i32::v2(4), str_or_i32::v2(2), str_or_i32::v2(3)], &[str_or_i32::v1("d2"), str_or_i32::v1("b222"), str_or_i32::v1("c222")], &[str_or_i32::v1("d"), str_or_i32::v1("b"), str_or_i32::v1("c")]] + let batch3 = create_batch_str_or_i32( + vec!["range", "v1", "hash1", "v2", "hash2"], + vec![ + &[ + str_or_i32::v1("range"), + str_or_i32::v1("range"), + str_or_i32::v1("range"), + ], + &[str_or_i32::v1("d1"), str_or_i32::v1("b111"), str_or_i32::v1("c111")], + &[str_or_i32::v2(4), str_or_i32::v2(2), str_or_i32::v2(3)], + &[str_or_i32::v1("d2"), str_or_i32::v1("b222"), str_or_i32::v1("c222")], + &[str_or_i32::v1("d"), str_or_i32::v1("b"), str_or_i32::v1("c")], + ], ); let builder = init_table( batch1, table_name, - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()]); + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], + ); - let builder = execute_upsert( - batch2, - table_name, - builder); + let builder = execute_upsert(batch2, table_name, builder); - let builder = execute_upsert( - batch3, - table_name, - builder); + let builder = execute_upsert(batch3, table_name, builder); check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["range", "hash1", "hash2", "v1", "v2"], @@ -948,18 +1367,21 @@ mod upsert_with_io_config_tests { "| range | 3 | c | c111 | c222 |", "| range | 4 | d | d1 | d2 |", "+-------+-------+-------+------+------+", - ] + ], ); check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["hash1", "v1", "v2"], @@ -974,18 +1396,21 @@ mod upsert_with_io_config_tests { "| 3 | c111 | c222 |", "| 4 | d1 | d2 |", "+-------+------+------+", - ] + ], ); check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["v1", "v2"], @@ -1000,18 +1425,21 @@ mod upsert_with_io_config_tests { "| c111 | c222 |", "| d1 | d2 |", "+------+------+", - ] + ], ); check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["range", "v2"], @@ -1026,34 +1454,53 @@ mod upsert_with_io_config_tests { "| range | c222 |", "| range | d2 |", "+-------+------+", - ] + ], ); } - #[test] - fn test_merge_same_column_with_timestamp_type_i32_time(){ - let dt1=NaiveDate::from_ymd_opt(1000, 6, 14).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt2=NaiveDate::from_ymd_opt(1582, 6, 15).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt3=NaiveDate::from_ymd_opt(1900, 6, 16).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt4=NaiveDate::from_ymd_opt(2018, 6, 17).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - - let val1=dt1.timestamp_micros(); - let val2=dt2.timestamp_micros(); - let val3=dt3.timestamp_micros(); - let val4=dt4.timestamp_micros(); + fn test_merge_same_column_with_timestamp_type_i32_time() { + let dt1 = NaiveDate::from_ymd_opt(1000, 6, 14) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt2 = NaiveDate::from_ymd_opt(1582, 6, 15) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt3 = NaiveDate::from_ymd_opt(1900, 6, 16) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt4 = NaiveDate::from_ymd_opt(2018, 6, 17) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + + let val1 = dt1.timestamp_micros(); + let val2 = dt2.timestamp_micros(); + let val3 = dt3.timestamp_micros(); + let val4 = dt4.timestamp_micros(); let table_name = "test_merge_same_column_with_timestamp_type_i64_time"; let builder = init_table( - create_batch_i32_and_timestamp(vec!["range", "hash", "value", "timestamp"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], vec![val1, val2, val3, val4]), - table_name, - vec!["range".to_string(), "hash".to_string()]); + create_batch_i32_and_timestamp( + vec!["range", "hash", "value", "timestamp"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + vec![val1, val2, val3, val4], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); check_upsert_i32_and_timestamp( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), - table_name, + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), + table_name, vec!["range", "hash", "value", "timestamp"], None, - builder.clone(), + builder.clone(), &[ "+----------+------+-------+----------------------------+", "| range | hash | value | timestamp |", @@ -1064,33 +1511,53 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | 44 | |", "| 20201102 | 4 | 4 | 2018-06-17T08:28:53.123456 |", "+----------+------+-------+----------------------------+", - ] + ], ); } #[test] - fn test_merge_different_columns_with_timestamp_type_i32_time(){ - let dt1=NaiveDate::from_ymd_opt(1000, 6, 14).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let _dt2=NaiveDate::from_ymd_opt(1582, 6, 15).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt3=NaiveDate::from_ymd_opt(1900, 6, 16).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt4=NaiveDate::from_ymd_opt(2018, 6, 17).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - - let val1=dt1.timestamp_micros(); - let _val2=_dt2.timestamp_micros(); - let val3=dt3.timestamp_micros(); - let val4=dt4.timestamp_micros(); + fn test_merge_different_columns_with_timestamp_type_i32_time() { + let dt1 = NaiveDate::from_ymd_opt(1000, 6, 14) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let _dt2 = NaiveDate::from_ymd_opt(1582, 6, 15) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt3 = NaiveDate::from_ymd_opt(1900, 6, 16) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt4 = NaiveDate::from_ymd_opt(2018, 6, 17) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + + let val1 = dt1.timestamp_micros(); + let _val2 = _dt2.timestamp_micros(); + let val3 = dt3.timestamp_micros(); + let val4 = dt4.timestamp_micros(); let table_name = "merge_different_columns_with_timestamp_type_i32_time"; let builder = init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - vec!["range".to_string(), "hash".to_string()]); + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + vec!["range".to_string(), "hash".to_string()], + ); check_upsert_i32_and_timestamp( - create_batch_i32_and_timestamp(vec!["range", "hash", "name", "timestamp"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]],vec![val1, val3, val4]), - table_name, + create_batch_i32_and_timestamp( + vec!["range", "hash", "name", "timestamp"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + vec![val1, val3, val4], + ), + table_name, vec!["range", "hash", "value", "name", "timestamp"], None, - builder.clone(), + builder.clone(), &[ "+----------+------+-------+------+----------------------------+", "| range | hash | value | name | timestamp |", @@ -1101,33 +1568,29 @@ mod upsert_with_io_config_tests { "| 20201101 | 4 | | 44 | 2018-06-17T08:28:53.123456 |", "| 20201102 | 4 | 4 | | |", "+----------+------+-------+------+----------------------------+", - ] + ], ); } - - } - mod upsert_with_metadata_tests { - - use std::sync::Arc; + use chrono::naive::NaiveDate; + use std::sync::Arc; use lakesoul_io::filter::parser::Parser; use arrow::datatypes::DataType; - - use arrow::datatypes::{Schema, SchemaRef, Field, TimeUnit}; - use arrow::record_batch::RecordBatch; use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampMicrosecondArray}; + use arrow::datatypes::{Field, Schema, SchemaRef, TimeUnit}; + use arrow::record_batch::RecordBatch; - use datafusion::assert_batches_eq; use crate::error::Result; use crate::lakesoul_table::LakeSoulTable; + use datafusion::assert_batches_eq; - use lakesoul_io::lakesoul_io_config::{LakeSoulIOConfigBuilder, create_session_context}; + use lakesoul_io::lakesoul_io_config::{create_session_context, LakeSoulIOConfigBuilder}; use lakesoul_metadata::{MetaDataClient, MetaDataClientRef}; @@ -1143,7 +1606,11 @@ mod upsert_with_metadata_tests { .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } @@ -1152,56 +1619,75 @@ mod upsert_with_metadata_tests { .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } - fn create_batch_i32_and_timestamp(names: Vec<&str>, values: Vec<&[i32]>, timestamp:Vec) -> RecordBatch { + fn create_batch_i32_and_timestamp(names: Vec<&str>, values: Vec<&[i32]>, timestamp: Vec) -> RecordBatch { let mut values = values .into_iter() .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) .collect::>(); let timestamp = Arc::new(TimestampMicrosecondArray::from(timestamp)) as ArrayRef; values.push(timestamp); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } fn create_batch_string(names: Vec<&str>, values: Vec<&[&str]>) -> RecordBatch { - let mut values=values + let mut values = values .into_iter() .map(|vec| Arc::new(StringArray::from(Vec::from(vec))) as ArrayRef) .collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } fn create_batch_str_or_i32(names: Vec<&str>, values: Vec<&[str_or_i32]>) -> RecordBatch { let values = values .into_iter() - .map(|vec| - match vec[0] { - str_or_i32::v1(_) => { - let vec=vec.into_iter().map(|val| - match val { - str_or_i32::v1(v1) => Some(*v1), - str_or_i32::v2(v2) => None, - } - ).map(|val| val.unwrap()).collect::>(); - Arc::new(StringArray::from(vec)) as ArrayRef - } - str_or_i32::v2(_) => { - let vec=vec.into_iter().map(|val| - match val { - str_or_i32::v1(v1) => None, - str_or_i32::v2(v2) => Some(v2), - } - ).map(|val| *val.unwrap()).collect::>(); - Arc::new(Int32Array::from(vec)) as ArrayRef - } + .map(|vec| match vec[0] { + str_or_i32::v1(_) => { + let vec = vec + .into_iter() + .map(|val| match val { + str_or_i32::v1(v1) => Some(*v1), + str_or_i32::v2(v2) => None, + }) + .map(|val| val.unwrap()) + .collect::>(); + Arc::new(StringArray::from(vec)) as ArrayRef + } + str_or_i32::v2(_) => { + let vec = vec + .into_iter() + .map(|val| match val { + str_or_i32::v1(v1) => None, + str_or_i32::v2(v2) => Some(v2), + }) + .map(|val| *val.unwrap()) + .collect::>(); + Arc::new(Int32Array::from(vec)) as ArrayRef } - ).collect::>(); - let iter = names.into_iter().zip(values).map(|(name, array)| (name, array, true)).collect::>(); + }) + .collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); RecordBatch::try_from_iter_with_nullable(iter).unwrap() } @@ -1210,13 +1696,19 @@ mod upsert_with_metadata_tests { lakesoul_table.execute_upsert(record_batch).await } - async fn check_upsert_debug(batch: RecordBatch, table_name: &str, selected_cols: Vec<&str>, filters: Option, client: MetaDataClientRef, expected: &[&str]) -> Result<()> { + async fn check_upsert_debug( + batch: RecordBatch, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + client: MetaDataClientRef, + expected: &[&str], + ) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; lakesoul_table.execute_upsert(batch).await?; let builder = create_io_config_builder(client, None, false).await?; let sess_ctx = create_session_context(&mut builder.clone().build())?; - - + let dataframe = lakesoul_table.to_dataframe(&sess_ctx).await?; let schema = SchemaRef::new(dataframe.schema().into()); @@ -1242,21 +1734,33 @@ mod upsert_with_metadata_tests { Ok(()) } - async fn init_table(batch: RecordBatch, table_name: &str, schema: SchemaRef, pks:Vec, client: MetaDataClientRef) -> Result<()> { + async fn init_table( + batch: RecordBatch, + table_name: &str, + schema: SchemaRef, + pks: Vec, + client: MetaDataClientRef, + ) -> Result<()> { let builder = LakeSoulIOConfigBuilder::new() - .with_schema(schema) - .with_primary_keys(pks); + .with_schema(schema) + .with_primary_keys(pks); create_table(client.clone(), table_name, builder.build()).await?; let lakesoul_table = LakeSoulTable::for_name(table_name).await?; lakesoul_table.execute_upsert(batch).await } - async fn check_upsert(batch: RecordBatch, table_name: &str, selected_cols: Vec<&str>, filters: Option, client: MetaDataClientRef, expected: &[&str]) -> Result<()> { + async fn check_upsert( + batch: RecordBatch, + table_name: &str, + selected_cols: Vec<&str>, + filters: Option, + client: MetaDataClientRef, + expected: &[&str], + ) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; lakesoul_table.execute_upsert(batch).await?; let builder = create_io_config_builder(client, None, false).await?; let sess_ctx = create_session_context(&mut builder.clone().build())?; - let dataframe = lakesoul_table.to_dataframe(&sess_ctx).await?; let schema = SchemaRef::new(dataframe.schema().into()); @@ -1273,11 +1777,11 @@ mod upsert_with_metadata_tests { dataframe.select_columns(&selected_cols)? }; - let result = dataframe - // .explain(true, false)? + let result = dataframe + // .explain(true, false)? .collect() .await?; - + // print_batches(&result); assert_batches_eq!(expected, &result); Ok(()) @@ -1287,17 +1791,29 @@ mod upsert_with_metadata_tests { let table_name = "test_merge_same_column_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), - vec!["range".to_string(), "hash".to_string()], - client.clone(), - ).await?; - + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash".to_string()], + client.clone(), + ) + .await?; + check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[4, 3, 1], &[44, 33, 11]]), - table_name, - vec!["range", "hash", "value"], + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[4, 3, 1], &[44, 33, 11]], + ), + table_name, + vec!["range", "hash", "value"], None, client.clone(), &[ @@ -1310,8 +1826,9 @@ mod upsert_with_metadata_tests { "| 20201101 | 4 | 44 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] - ).await + ], + ) + .await } async fn test_merge_different_column_i32() -> Result<()> { @@ -1319,18 +1836,30 @@ mod upsert_with_metadata_tests { let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; - + ) + .await?; + check_upsert( - create_batch_i32(vec!["range", "hash", "name"], vec![&[20201101, 20201101, 20201101], &[1, 4, 3], &[11, 44, 33]]), - table_name, - vec!["range", "hash", "value", "name"], - None, + create_batch_i32( + vec!["range", "hash", "name"], + vec![&[20201101, 20201101, 20201101], &[1, 4, 3], &[11, 44, 33]], + ), + table_name, + vec!["range", "hash", "value", "name"], + None, client.clone(), &[ "+----------+------+-------+------+", @@ -1342,8 +1871,9 @@ mod upsert_with_metadata_tests { "| 20201101 | 4 | | 44 |", "| 20201102 | 4 | 4 | |", "+----------+------+-------+------+", - ] - ).await + ], + ) + .await } async fn test_merge_different_columns_and_filter_by_non_selected_columns_i32() -> Result<()> { @@ -1351,17 +1881,29 @@ mod upsert_with_metadata_tests { let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "name"], vec![&[20201101, 20201101, 20201101], &[3, 1, 4], &[33, 11, 44]]), - table_name, - vec!["range", "hash", "value"], + create_batch_i32( + vec!["range", "hash", "name"], + vec![&[20201101, 20201101, 20201101], &[3, 1, 4], &[33, 11, 44]], + ), + table_name, + vec!["range", "hash", "value"], Some("and(noteq(name, null), gt(name, 0))".to_string()), client.clone(), &[ @@ -1372,25 +1914,42 @@ mod upsert_with_metadata_tests { "| 20201101 | 3 | 3 |", "| 20201101 | 4 | |", "+----------+------+-------+", - ] - ).await - + ], + ) + .await } - async fn test_merge_different_columns_and_filter_partial_rows_i32() -> Result<()>{ + async fn test_merge_different_columns_and_filter_partial_rows_i32() -> Result<()> { let table_name = "merge-different_columns_and_filter_partial_rows_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4], &[11, 22, 33, 44]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201101, 20201101, 20201101, 20201102], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[11, 22, 33, 44], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 3, 4, 4], &[2, 4, 5, 5]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 3, 4, 4], &[2, 4, 5, 5]], + ), table_name, vec!["range", "hash", "value", "name"], Some("and(and(noteq(value, null), lt(value, 5)),and(noteq(name, null), gt(name, 0)))".to_string()), @@ -1403,25 +1962,38 @@ mod upsert_with_metadata_tests { "| 20201101 | 2 | 2 | 22 |", "| 20201101 | 3 | 4 | 33 |", "+----------+------+-------+------+", - ] - ).await + ], + ) + .await } - async fn test_merge_one_file_with_empty_batch_i32() -> Result<()>{ + async fn test_merge_one_file_with_empty_batch_i32() -> Result<()> { let table_name = "merge_one_file_with_empty_batch"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), ))), table_name, vec!["range", "hash", "value"], @@ -1434,31 +2006,49 @@ mod upsert_with_metadata_tests { "| 20201101 | 1 | 1 |", "| 20201101 | 2 | 2 |", "+----------+------+-------+", - ] - ).await + ], + ) + .await } - async fn test_merge_multi_files_with_empty_batch_i32() -> Result<()>{ + async fn test_merge_multi_files_with_empty_batch_i32() -> Result<()> { let table_name = "merge_multi_files_with_empty_batch"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102, 20201102], &[1, 2, 3, 4, 1], &[1, 2, 3, 4, 1]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![ + &[20201101, 20201101, 20201101, 20201102, 20201102], + &[1, 2, 3, 4, 1], + &[1, 2, 3, 4, 1], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102], &[4], &[5]]), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), ))), table_name, vec!["range", "hash", "value"], @@ -1472,27 +2062,39 @@ mod upsert_with_metadata_tests { "| 20201101 | 2 | 2 |", "| 20201102 | 1 | 1 |", "+----------+------+-------+", - ] - ).await + ], + ) + .await } - - async fn test_basic_upsert_same_columns() -> Result<()>{ + async fn test_basic_upsert_same_columns() -> Result<()> { // require metadata checker let table_name = "test_basic_upsert_same_columns"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), ))), table_name, vec!["range", "hash", "value"], @@ -1507,11 +2109,15 @@ mod upsert_with_metadata_tests { "| 20201101 | 3 | 3 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] - ).await?; + ], + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value"], None, @@ -1526,26 +2132,39 @@ mod upsert_with_metadata_tests { "| 20201101 | 4 | 44 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] - ).await + ], + ) + .await } - async fn test_basic_upsert_different_columns() -> Result<()>{ + async fn test_basic_upsert_different_columns() -> Result<()> { // require metadata checker let table_name = "test_basic_upsert_same_columns"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"].iter().map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)).collect::>() + vec!["range", "hash", "value"] + .iter() + .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .collect::>(), ))), table_name, vec!["range", "hash", "value"], @@ -1560,11 +2179,15 @@ mod upsert_with_metadata_tests { "| 20201101 | 3 | 3 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] - ).await?; + ], + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value"], None, @@ -1572,46 +2195,59 @@ mod upsert_with_metadata_tests { &[ "+----------+------+-------+", "| range | hash | value |", - "+----------+------+-------+", + "+----------+------+-------+", "| 20201101 | 1 | 11 |", "| 20201101 | 2 | 2 |", "| 20201101 | 3 | 33 |", - "| 20201101 | 4 | 44 |", + "| 20201101 | 4 | 44 |", "| 20201102 | 4 | 4 |", "+----------+------+-------+", - ] - ).await + ], + ) + .await } - async fn test_should_failed_to_upsert_external_columns_when_schema_auto_migrate_is_false() -> Result<()>{ + async fn test_should_failed_to_upsert_external_columns_when_schema_auto_migrate_is_false() -> Result<()> { // require metadata checker Ok(()) } - async fn test_upsert_in_new_table_should_failed() -> Result<()>{ + async fn test_upsert_in_new_table_should_failed() -> Result<()> { // require metadata checker Ok(()) } - async fn test_upsert_cant_use_delta_file() -> Result<()>{ + async fn test_upsert_cant_use_delta_file() -> Result<()> { // require metadata checker Ok(()) } - async fn test_upsert_without_range_parqitions_i32() -> Result<()>{ + async fn test_upsert_without_range_parqitions_i32() -> Result<()> { let table_name = "upsert_without_range_parqitions"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value"], None, @@ -1625,29 +2261,59 @@ mod upsert_with_metadata_tests { "| 20201101 | 3 | 33 |", "| 20201101 | 4 | 44 |", "+----------+------+-------+", - ] - ).await + ], + ) + .await } - async fn test_upsert_without_hash_partitions_should_fail() -> Result<()>{ + async fn test_upsert_without_hash_partitions_should_fail() -> Result<()> { // require metadata checker Ok(()) } - async fn test_upsert_with_multiple_range_and_hash_parqitions_i32() -> Result<()>{ + async fn test_upsert_with_multiple_range_and_hash_parqitions_i32() -> Result<()> { let table_name = "upsert_with_multiple_range_and_hash_parqitions"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range1", "range2", "hash1", "hash2", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range1", "range2", "hash1", "hash2", "value"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), - vec!["range1".to_string(), "range2".to_string(), "hash1".to_string(), "hash2".to_string()], + create_batch_i32( + vec!["range1", "range2", "hash1", "hash2", "value"], + vec![ + &[20201101, 20201101, 20201101, 20201102], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range1", "range2", "hash1", "hash2", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), + vec![ + "range1".to_string(), + "range2".to_string(), + "hash1".to_string(), + "hash2".to_string(), + ], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range1", "range2", "hash1", "hash2", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[1, 3, 4],&[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range1", "range2", "hash1", "hash2", "value"], + vec![ + &[20201101, 20201101, 20201101], + &[1, 3, 4], + &[1, 3, 4], + &[1, 3, 4], + &[11, 33, 44], + ], + ), table_name, vec!["range1", "range2", "hash1", "hash2", "value"], None, @@ -1662,34 +2328,53 @@ mod upsert_with_metadata_tests { "| 20201101 | 4 | 4 | 4 | 44 |", "| 20201102 | 4 | 4 | 4 | 4 |", "+----------+--------+-------+-------+-------+", - ] - ).await + ], + ) + .await } - async fn test_source_dataframe_without_partition_columns() -> Result<()>{ + async fn test_source_dataframe_without_partition_columns() -> Result<()> { // require metadata checker Ok(()) } - async fn test_upsert_with_condition() -> Result<()>{ + async fn test_upsert_with_condition() -> Result<()> { // require metadata checker Ok(()) } - async fn test_filter_requested_columns_upsert_1_times_i32() -> Result<()>{ + async fn test_filter_requested_columns_upsert_1_times_i32() -> Result<()> { let table_name = "filter_requested_columns_upsert_1_times"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101, 20201101, 20201101], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![ + &[20201101, 20201101, 20201101, 20201101], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), @@ -1702,30 +2387,58 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 33 | | |", "| 20201102 | 4 | 44 | | |", "+----------+------+-------+------+-----+", - ] - ).await + ], + ) + .await } - async fn test_filter_requested_columns_upsert_2_times_i32() -> Result<()>{ + async fn test_filter_requested_columns_upsert_2_times_i32() -> Result<()> { let table_name = "filter_requested_columns_upsert_2_times"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101, 20201101, 20201101], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![ + &[20201101, 20201101, 20201101, 20201101], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201102, 20201102, 20201102], &[1, 2, 3], &[111, 222, 333], &[11, 22, 33]]), + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201102, 20201102, 20201102], + &[1, 2, 3], + &[111, 222, 333], + &[11, 22, 33], + ], + ), table_name, vec!["range", "hash", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), @@ -1739,43 +2452,72 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 333 | 33 | |", "| 20201102 | 4 | 44 | | |", "+----------+------+-------+------+-----+", - ] - ).await + ], + ) + .await } - async fn test_filter_requested_columns_upsert_3_times_i32() -> Result<()>{ + async fn test_filter_requested_columns_upsert_3_times_i32() -> Result<()> { let table_name = "filter_requested_columns_upsert_3_times"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101, 20201101, 20201101], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![ + &[20201101, 20201101, 20201101, 20201101], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + &[1, 2, 3, 4], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201102, 20201102, 20201102], &[1, 2, 3], &[111, 222, 333], &[11, 22, 33]]), + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201102, 20201102, 20201102], + &[1, 2, 3], + &[111, 222, 333], + &[11, 22, 33], + ], + ), table_name, client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), table_name, client.clone(), - ).await?; + ) + .await?; /*** - !!! Error all below conditions are satisfied - 1. entire 'value' column is null - 2. filter exists - 3. filter pushed down into TableProvider with TableProviderFilterPushDown::Inexact - 4. SessionConfig.execution.parquet.pruning = true (equivalent SessionConfig.with_parquet_pruning(true)) - 5. SessionConfig.execution.parquet.enable_page_index = true - 6. - */ + !!! Error all below conditions are satisfied + 1. entire 'value' column is null + 2. filter exists + 3. filter pushed down into TableProvider with TableProviderFilterPushDown::Inexact + 4. SessionConfig.execution.parquet.pruning = true (equivalent SessionConfig.with_parquet_pruning(true)) + 5. SessionConfig.execution.parquet.enable_page_index = true + 6. + */ check_upsert( create_batch_optional_i32( vec!["range", "hash", "age", "name", "value"], @@ -1784,8 +2526,10 @@ mod upsert_with_metadata_tests { &[Some(1), Some(3)], &[Some(111), Some(333)], &[Some(11), Some(33)], - &[None, Some(3333)]]), - // &[None, None]]), + &[None, Some(3333)], + ], + ), + // &[None, None]]), table_name, vec!["range", "hash", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), @@ -1801,96 +2545,127 @@ mod upsert_with_metadata_tests { // "| 20201102 | 3 | | 33 | 333 |", "| 20201102 | 4 | 44 | | |", "+----------+------+-------+------+-----+", - ] - ).await - + ], + ) + .await } - async fn test_select_requested_columns_without_hash_columns_upsert_1_times_i32() -> Result<()>{ + async fn test_select_requested_columns_without_hash_columns_upsert_1_times_i32() -> Result<()> { let table_name = "select_requested_columns_without_hash_columns_upsert_1_times"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["age"], None, client.clone(), &[ - "+-----+", - "| age |", - "+-----+", - "| 1 |", - "| 2 |", - "| |", - "| |", - "| |", - "+-----+", - ] - ).await + "+-----+", "| age |", "+-----+", "| 1 |", "| 2 |", "| |", "| |", "| |", "+-----+", + ], + ) + .await } - async fn test_select_requested_columns_without_hash_columns_upsert_2_times_i32() -> Result<()>{ + async fn test_select_requested_columns_without_hash_columns_upsert_2_times_i32() -> Result<()> { let table_name = "select_requested_columns_without_hash_columns_upsert_2_times"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), + create_batch_i32( + vec!["range", "hash", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[11, 33, 44]], + ), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value", "name"], vec![&[20201102, 20201102, 20201102], &[1, 2, 3], &[111, 222, 333], &[11, 22, 33]]), + create_batch_i32( + vec!["range", "hash", "value", "name"], + vec![ + &[20201102, 20201102, 20201102], + &[1, 2, 3], + &[111, 222, 333], + &[11, 22, 33], + ], + ), table_name, vec!["age"], None, client.clone(), &[ + "+-----+", "| age |", "+-----+", "| 1 |", "| 2 |", "| |", "| |", "| |", "| |", "+-----+", - "| age |", - "+-----+", - "| 1 |", - "| 2 |", - "| |", - "| |", - "| |", - "| |", - "+-----+", - ] - ).await + ], + ) + .await } - async fn test_derange_hash_key_and_data_schema_order_int_type_upsert_1_times_i32() -> Result<()>{ + async fn test_derange_hash_key_and_data_schema_order_int_type_upsert_1_times_i32() -> Result<()> { let table_name = "derange_hash_key_and_data_schema_order_int_type_upsert_1_times_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2],&[1, 2]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash1", "hash2", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + create_batch_i32( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash1", "hash2", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash1", "hash2", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]]), + create_batch_i32( + vec!["range", "hash1", "hash2", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]], + ), table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), @@ -1903,30 +2678,52 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 32 | 3 | | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] - ).await + ], + ) + .await } - async fn test_derange_hash_key_and_data_schema_order_int_type_upsert_2_times_i32() -> Result<()>{ + async fn test_derange_hash_key_and_data_schema_order_int_type_upsert_2_times_i32() -> Result<()> { let table_name = "derange_hash_key_and_data_schema_order_int_type_upsert_2_times_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2],&[1, 2]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash1", "hash2", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + create_batch_i32( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash1", "hash2", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash1", "hash2", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]]), + create_batch_i32( + vec!["range", "hash1", "hash2", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]], + ), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash2", "name", "hash1"], vec![&[20201102, 20201102, 20201102], &[12, 22, 32], &[11, 22, 33], &[1, 2, 3]]), + create_batch_i32( + vec!["range", "hash2", "name", "hash1"], + vec![ + &[20201102, 20201102, 20201102], + &[12, 22, 32], + &[11, 22, 33], + &[1, 2, 3], + ], + ), table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, 20201102))".to_string()), @@ -1940,36 +2737,68 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 32 | 3 | 33 | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] - ).await + ], + ) + .await } - async fn test_derange_hash_key_and_data_schema_order_int_type_upsert_3_times_i32() -> Result<()>{ + async fn test_derange_hash_key_and_data_schema_order_int_type_upsert_3_times_i32() -> Result<()> { let table_name = "derange_hash_key_and_data_schema_order_int_type_upsert_3_times_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2],&[1, 2]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash1", "hash2", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Int32, true)).collect::>())), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + create_batch_i32( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![&[20201101, 20201101], &[1, 2], &[1, 2], &[1, 2], &[1, 2], &[1, 2]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash1", "hash2", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash1", "hash2", "value"], vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]]), + create_batch_i32( + vec!["range", "hash1", "hash2", "value"], + vec![&[20201102, 20201102, 20201102], &[1, 3, 4], &[12, 32, 42], &[1, 3, 4]], + ), table_name, client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_i32(vec!["range", "hash2", "name", "hash1"], vec![&[20201102, 20201102, 20201102], &[12, 22, 32], &[11, 22, 33], &[1, 2, 3]]), + create_batch_i32( + vec!["range", "hash2", "name", "hash1"], + vec![ + &[20201102, 20201102, 20201102], + &[12, 22, 32], + &[11, 22, 33], + &[1, 2, 3], + ], + ), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert_debug( - create_batch_i32(vec!["range", "age", "hash2", "name", "hash1"], vec![&[20201102, 20201102, 20201102], &[4567, 2345, 3456], &[42, 22, 32], &[456, 234, 345], &[4, 2, 3]]), + create_batch_i32( + vec!["range", "age", "hash2", "name", "hash1"], + vec![ + &[20201102, 20201102, 20201102], + &[4567, 2345, 3456], + &[42, 22, 32], + &[456, 234, 345], + &[4, 2, 3], + ], + ), table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(and(noteq(range, null), eq(range, 20201102)), noteq(value, null))".to_string()), @@ -1982,24 +2811,49 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 32 | 3 | 345 | 3456 |", "| 20201102 | 4 | 42 | 4 | 456 | 4567 |", "+----------+-------+-------+-------+------+------+", - ] - ).await + ], + ) + .await } - async fn test_derange_hash_key_and_data_schema_order_string_type_upsert_1_times_i32() -> Result<()>{ + async fn test_derange_hash_key_and_data_schema_order_string_type_upsert_1_times_i32() -> Result<()> { let table_name = "derange_hash_key_and_data_schema_order_string_type_upsert_1_times_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_string(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&["20201101", "20201101"], &["1", "2"], &["1", "2"], &["1", "2"], &["1", "2"],&["1", "2"]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash1", "hash2", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Utf8, true)).collect::>())), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + create_batch_string( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![ + &["20201101", "20201101"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash1", "hash2", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Utf8, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_string(vec!["range", "hash1", "hash2", "value"], vec![&["20201102", "20201102", "20201102"], &["1", "3", "4"], &["12", "32", "42"], &["1", "3", "4"]]), + create_batch_string( + vec!["range", "hash1", "hash2", "value"], + vec![ + &["20201102", "20201102", "20201102"], + &["1", "3", "4"], + &["12", "32", "42"], + &["1", "3", "4"], + ], + ), table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, ________20201102__))".to_string()), @@ -2012,30 +2866,64 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 32 | 3 | | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] - ).await + ], + ) + .await } - async fn test_derange_hash_key_and_data_schema_order_string_type_upsert_2_times_i32() -> Result<()>{ + async fn test_derange_hash_key_and_data_schema_order_string_type_upsert_2_times_i32() -> Result<()> { let table_name = "derange_hash_key_and_data_schema_order_string_type_upsert_2_times_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_string(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&["20201101", "20201101"], &["1", "2"], &["1", "2"], &["1", "2"], &["1", "2"],&["1", "2"]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash1", "hash2", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Utf8, true)).collect::>())), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + create_batch_string( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![ + &["20201101", "20201101"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash1", "hash2", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Utf8, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_string(vec!["range", "hash1", "hash2", "value"], vec![&["20201102", "20201102", "20201102"], &["1", "3", "4"], &["12", "32", "42"], &["1", "3", "4"]]), + create_batch_string( + vec!["range", "hash1", "hash2", "value"], + vec![ + &["20201102", "20201102", "20201102"], + &["1", "3", "4"], + &["12", "32", "42"], + &["1", "3", "4"], + ], + ), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_string(vec!["range", "hash2", "name", "hash1"], vec![&["20201102", "20201102", "20201102"], &["12", "22", "32"], &["11", "22", "33"], &["1", "2", "3"]]), + create_batch_string( + vec!["range", "hash2", "name", "hash1"], + vec![ + &["20201102", "20201102", "20201102"], + &["12", "22", "32"], + &["11", "22", "33"], + &["1", "2", "3"], + ], + ), table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(noteq(range, null), eq(range, ________20201102__))".to_string()), @@ -2049,36 +2937,80 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 32 | 3 | 33 | |", "| 20201102 | 4 | 42 | 4 | | |", "+----------+-------+-------+-------+------+-----+", - ] - ).await + ], + ) + .await } - async fn test_derange_hash_key_and_data_schema_order_string_type_upsert_3_times_i32() -> Result<()>{ + async fn test_derange_hash_key_and_data_schema_order_string_type_upsert_3_times_i32() -> Result<()> { let table_name = "derange_hash_key_and_data_schema_order_string_type_upsert_3_times_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_string(vec!["range", "hash1", "hash2", "value", "name", "age"], vec![&["20201101", "20201101"], &["1", "2"], &["1", "2"], &["1", "2"], &["1", "2"],&["1", "2"]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash1", "hash2", "value", "name", "age"].into_iter().map(|name| Field::new(name, DataType::Utf8, true)).collect::>())), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + create_batch_string( + vec!["range", "hash1", "hash2", "value", "name", "age"], + vec![ + &["20201101", "20201101"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + &["1", "2"], + ], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash1", "hash2", "value", "name", "age"] + .into_iter() + .map(|name| Field::new(name, DataType::Utf8, true)) + .collect::>(), + )), + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_string(vec!["range", "hash1", "hash2", "value"], vec![&["20201102", "20201102", "20201102"], &["1", "3", "4"], &["12", "32", "42"], &["1", "3", "4"]]), + create_batch_string( + vec!["range", "hash1", "hash2", "value"], + vec![ + &["20201102", "20201102", "20201102"], + &["1", "3", "4"], + &["12", "32", "42"], + &["1", "3", "4"], + ], + ), table_name, client.clone(), - ).await?; + ) + .await?; execute_upsert( - create_batch_string(vec!["range", "hash2", "name", "hash1"], vec![&["20201102", "20201102", "20201102"], &["12", "22", "32"], &["11", "22", "33"], &["1", "2", "3"]]), + create_batch_string( + vec!["range", "hash2", "name", "hash1"], + vec![ + &["20201102", "20201102", "20201102"], + &["12", "22", "32"], + &["11", "22", "33"], + &["1", "2", "3"], + ], + ), table_name, client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_string(vec!["range", "age", "hash2", "name", "hash1"], vec![&["20201102", "20201102", "20201102"], &["4567", "2345", "3456"], &["42", "22", "32"], &["456", "234", "345"], &["4", "2", "3"]]), + create_batch_string( + vec!["range", "age", "hash2", "name", "hash1"], + vec![ + &["20201102", "20201102", "20201102"], + &["4567", "2345", "3456"], + &["42", "22", "32"], + &["456", "234", "345"], + &["4", "2", "3"], + ], + ), table_name, vec!["range", "hash1", "hash2", "value", "name", "age"], Some("and(and(noteq(range, null), eq(range, ________20201102__)), noteq(value, null))".to_string()), @@ -2091,65 +3023,96 @@ mod upsert_with_metadata_tests { "| 20201102 | 3 | 32 | 3 | 345 | 3456 |", "| 20201102 | 4 | 42 | 4 | 456 | 4567 |", "+----------+-------+-------+-------+------+------+", - ] - ).await + ], + ) + .await } - async fn test_create_table_with_hash_key_disordered() -> Result<()>{ + async fn test_create_table_with_hash_key_disordered() -> Result<()> { let table_name = "test_create_table_with_hash_key_disordered"; let client = Arc::new(MetaDataClient::from_env().await?); - let batch1=create_batch_str_or_i32(vec!["range", "v1", "hash1", "v2", "hash2"], - vec![&[str_or_i32::v1("range"), str_or_i32::v1("range"), str_or_i32::v1("range")], &[str_or_i32::v1("a1"), str_or_i32::v1("b1"), str_or_i32::v1("c1")], - &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], &[str_or_i32::v1("a2"), str_or_i32::v1("b2"), str_or_i32::v1("c2")], &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")]] + let batch1 = create_batch_str_or_i32( + vec!["range", "v1", "hash1", "v2", "hash2"], + vec![ + &[ + str_or_i32::v1("range"), + str_or_i32::v1("range"), + str_or_i32::v1("range"), + ], + &[str_or_i32::v1("a1"), str_or_i32::v1("b1"), str_or_i32::v1("c1")], + &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], + &[str_or_i32::v1("a2"), str_or_i32::v1("b2"), str_or_i32::v1("c2")], + &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + ], ); - let batch2=create_batch_str_or_i32(vec!["range", "hash1", "v1", "v2", "hash2"], - vec![&[str_or_i32::v1("range"), str_or_i32::v1("range"), str_or_i32::v1("range")], &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], &[str_or_i32::v1("a11"), str_or_i32::v1("b11"), str_or_i32::v1("c11")], - &[str_or_i32::v1("a22"), str_or_i32::v1("b22"), str_or_i32::v1("c22")], &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")]] + let batch2 = create_batch_str_or_i32( + vec!["range", "hash1", "v1", "v2", "hash2"], + vec![ + &[ + str_or_i32::v1("range"), + str_or_i32::v1("range"), + str_or_i32::v1("range"), + ], + &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], + &[str_or_i32::v1("a11"), str_or_i32::v1("b11"), str_or_i32::v1("c11")], + &[str_or_i32::v1("a22"), str_or_i32::v1("b22"), str_or_i32::v1("c22")], + &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + ], ); - let batch3=create_batch_str_or_i32(vec!["range", "v1", "hash1", "v2", "hash2"], - vec![&[str_or_i32::v1("range"), str_or_i32::v1("range"), str_or_i32::v1("range")], &[str_or_i32::v1("d1"), str_or_i32::v1("b111"), str_or_i32::v1("c111")], - &[str_or_i32::v2(4), str_or_i32::v2(2), str_or_i32::v2(3)], &[str_or_i32::v1("d2"), str_or_i32::v1("b222"), str_or_i32::v1("c222")], &[str_or_i32::v1("d"), str_or_i32::v1("b"), str_or_i32::v1("c")]] + let batch3 = create_batch_str_or_i32( + vec!["range", "v1", "hash1", "v2", "hash2"], + vec![ + &[ + str_or_i32::v1("range"), + str_or_i32::v1("range"), + str_or_i32::v1("range"), + ], + &[str_or_i32::v1("d1"), str_or_i32::v1("b111"), str_or_i32::v1("c111")], + &[str_or_i32::v2(4), str_or_i32::v2(2), str_or_i32::v2(3)], + &[str_or_i32::v1("d2"), str_or_i32::v1("b222"), str_or_i32::v1("c222")], + &[str_or_i32::v1("d"), str_or_i32::v1("b"), str_or_i32::v1("c")], + ], ); init_table( batch1, table_name, - SchemaRef::new(Schema::new(["range", "v1", "hash1", "v2", "hash2"].into_iter().map(|name| - if name=="hash1"{ - Field::new(name, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(name, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + SchemaRef::new(Schema::new( + ["range", "v1", "hash1", "v2", "hash2"] + .into_iter() + .map(|name| { + if name == "hash1" { + Field::new(name, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(name, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), )), - vec!["range".to_string(), "hash1".to_string(),"hash2".to_string()], + vec!["range".to_string(), "hash1".to_string(), "hash2".to_string()], client.clone(), - ).await?; + ) + .await?; - execute_upsert( - batch2, - table_name, - client.clone(), - ).await?; + execute_upsert(batch2, table_name, client.clone()).await?; - execute_upsert( - batch3, - table_name, - client.clone(), - ).await?; + execute_upsert(batch3, table_name, client.clone()).await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["range", "hash1", "hash2", "v1", "v2"], @@ -2164,18 +3127,22 @@ mod upsert_with_metadata_tests { "| range | 3 | c | c111 | c222 |", "| range | 4 | d | d1 | d2 |", "+-------+-------+-------+------+------+", - ] - ).await?; + ], + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["hash1", "v1", "v2"], @@ -2190,18 +3157,22 @@ mod upsert_with_metadata_tests { "| 3 | c111 | c222 |", "| 4 | d1 | d2 |", "+-------+------+------+", - ] - ).await?; + ], + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["v1", "v2"], @@ -2216,18 +3187,22 @@ mod upsert_with_metadata_tests { "| c111 | c222 |", "| d1 | d2 |", "+------+------+", - ] - ).await?; + ], + ) + .await?; check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"].iter().map(|col| - if *col=="hash1"{ - Field::new(*col, arrow::datatypes::DataType::Int32, true) - }else{ - Field::new(*col, arrow::datatypes::DataType::Utf8, true) - } - ).collect::>() + vec!["range", "v1", "hash1", "v2", "hash2"] + .iter() + .map(|col| { + if *col == "hash1" { + Field::new(*col, arrow::datatypes::DataType::Int32, true) + } else { + Field::new(*col, arrow::datatypes::DataType::Utf8, true) + } + }) + .collect::>(), ))), table_name, vec!["range", "v2"], @@ -2242,40 +3217,66 @@ mod upsert_with_metadata_tests { "| range | c222 |", "| range | d2 |", "+-------+------+", - ] - ).await - } - - async fn test_merge_same_column_with_timestamp_type_i32_time() -> Result<()>{ - let dt1=NaiveDate::from_ymd_opt(1000, 6, 14).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt2=NaiveDate::from_ymd_opt(1582, 6, 15).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt3=NaiveDate::from_ymd_opt(1900, 6, 16).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt4=NaiveDate::from_ymd_opt(2018, 6, 17).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - - let val1=dt1.timestamp_micros(); - let val2=dt2.timestamp_micros(); - let val3=dt3.timestamp_micros(); - let val4=dt4.timestamp_micros(); + ], + ) + .await + } + + async fn test_merge_same_column_with_timestamp_type_i32_time() -> Result<()> { + let dt1 = NaiveDate::from_ymd_opt(1000, 6, 14) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt2 = NaiveDate::from_ymd_opt(1582, 6, 15) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt3 = NaiveDate::from_ymd_opt(1900, 6, 16) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt4 = NaiveDate::from_ymd_opt(2018, 6, 17) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + + let val1 = dt1.timestamp_micros(); + let val2 = dt2.timestamp_micros(); + let val3 = dt3.timestamp_micros(); + let val4 = dt4.timestamp_micros(); let table_name = "test_merge_same_column_with_timestamp_type_i64_time"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( - create_batch_i32_and_timestamp(vec!["range", "hash", "value", "timestamp"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], vec![val1, val2, val3, val4]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "timestamp"].into_iter().map(|name| - if name=="timestamp" { - Field::new(name, DataType::Timestamp(TimeUnit::Microsecond, None), true) - }else { - Field::new(name, DataType::Int32, true) - } - ).collect::>())), + create_batch_i32_and_timestamp( + vec!["range", "hash", "value", "timestamp"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + vec![val1, val2, val3, val4], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "timestamp"] + .into_iter() + .map(|name| { + if name == "timestamp" { + Field::new(name, DataType::Timestamp(TimeUnit::Microsecond, None), true) + } else { + Field::new(name, DataType::Int32, true) + } + }) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]]), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + ), table_name, vec!["range", "hash", "value", "timestamp"], None, @@ -2290,41 +3291,66 @@ mod upsert_with_metadata_tests { "| 20201101 | 4 | 44 | |", "| 20201102 | 4 | 4 | 2018-06-17T08:28:53.123456 |", "+----------+------+-------+----------------------------+", - ] - ).await - } - - async fn test_merge_different_columns_with_timestamp_type_i32_time() -> Result<()>{ - let dt1=NaiveDate::from_ymd_opt(1000, 6, 14).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let _dt2=NaiveDate::from_ymd_opt(1582, 6, 15).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt3=NaiveDate::from_ymd_opt(1900, 6, 16).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - let dt4=NaiveDate::from_ymd_opt(2018, 6, 17).unwrap().and_hms_micro_opt(8, 28, 53, 123456).unwrap(); - - let val1=dt1.timestamp_micros(); - let _val2=_dt2.timestamp_micros(); - let val3=dt3.timestamp_micros(); - let val4=dt4.timestamp_micros(); + ], + ) + .await + } + + async fn test_merge_different_columns_with_timestamp_type_i32_time() -> Result<()> { + let dt1 = NaiveDate::from_ymd_opt(1000, 6, 14) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let _dt2 = NaiveDate::from_ymd_opt(1582, 6, 15) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt3 = NaiveDate::from_ymd_opt(1900, 6, 16) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + let dt4 = NaiveDate::from_ymd_opt(2018, 6, 17) + .unwrap() + .and_hms_micro_opt(8, 28, 53, 123456) + .unwrap(); + + let val1 = dt1.timestamp_micros(); + let _val2 = _dt2.timestamp_micros(); + let val3 = dt3.timestamp_micros(); + let val4 = dt4.timestamp_micros(); let table_name = "merge_different_columns_with_timestamp_type_i32_time"; let client = Arc::new(MetaDataClient::from_env().await?); - init_table( - create_batch_i32(vec!["range", "hash", "value"], vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]]), - table_name, - SchemaRef::new(Schema::new(["range", "hash", "value", "name", "timestamp"].into_iter().map(|name| - if name=="timestamp" { - Field::new(name, DataType::Timestamp(TimeUnit::Microsecond, None), true) - }else { - Field::new(name, DataType::Int32, true) - } - ).collect::>())), + create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ), + table_name, + SchemaRef::new(Schema::new( + ["range", "hash", "value", "name", "timestamp"] + .into_iter() + .map(|name| { + if name == "timestamp" { + Field::new(name, DataType::Timestamp(TimeUnit::Microsecond, None), true) + } else { + Field::new(name, DataType::Int32, true) + } + }) + .collect::>(), + )), vec!["range".to_string(), "hash".to_string()], client.clone(), - ).await?; + ) + .await?; check_upsert( - create_batch_i32_and_timestamp(vec!["range", "hash", "name", "timestamp"], vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]],vec![val1, val3, val4]), + create_batch_i32_and_timestamp( + vec!["range", "hash", "name", "timestamp"], + vec![&[20201101, 20201101, 20201101], &[1, 3, 4], &[11, 33, 44]], + vec![val1, val3, val4], + ), table_name, vec!["range", "hash", "value", "name", "timestamp"], None, @@ -2339,12 +3365,13 @@ mod upsert_with_metadata_tests { "| 20201101 | 4 | | 44 | 2018-06-17T08:28:53.123456 |", "| 20201102 | 4 | 4 | | |", "+----------+------+-------+------+----------------------------+", - ] - ).await + ], + ) + .await } #[tokio::test] - async fn test_all_cases() -> Result<()> { + async fn test_all_cases() -> Result<()> { test_merge_same_column_i32().await?; test_merge_different_column_i32().await?; test_merge_different_columns_and_filter_by_non_selected_columns_i32().await?; @@ -2370,6 +3397,4 @@ mod upsert_with_metadata_tests { Ok(()) } - } - diff --git a/rust/lakesoul-io-c/build.rs b/rust/lakesoul-io-c/build.rs index ea3a588fe..7104c839d 100644 --- a/rust/lakesoul-io-c/build.rs +++ b/rust/lakesoul-io-c/build.rs @@ -14,4 +14,4 @@ fn main() { .generate() .expect("Unable to generate bindings") .write_to_file("lakesoul_c_bindings.h"); -} \ No newline at end of file +} diff --git a/rust/lakesoul-io/src/datasource/empty_schema.rs b/rust/lakesoul-io/src/datasource/empty_schema.rs index 33ac32438..5c52c44d6 100644 --- a/rust/lakesoul-io/src/datasource/empty_schema.rs +++ b/rust/lakesoul-io/src/datasource/empty_schema.rs @@ -5,18 +5,17 @@ use std::any::Any; use std::sync::Arc; -use arrow::datatypes::{SchemaRef, Schema}; +use arrow::datatypes::{Schema, SchemaRef}; use async_trait::async_trait; +use datafusion::execution::context::SessionState; use datafusion::logical_expr::Expr; use datafusion::physical_plan::ExecutionPlan; use datafusion::{datasource::TableProvider, logical_expr::TableType}; -use datafusion::execution::context::SessionState; use datafusion_common::Result; use super::physical_plan::EmptySchemaScanExec; - #[derive(Clone, Debug)] pub struct EmptySchemaProvider { count: usize, diff --git a/rust/lakesoul-io/src/datasource/file_format.rs b/rust/lakesoul-io/src/datasource/file_format.rs index 01e8f6da6..b9579d8ae 100644 --- a/rust/lakesoul-io/src/datasource/file_format.rs +++ b/rust/lakesoul-io/src/datasource/file_format.rs @@ -2,22 +2,22 @@ // // SPDX-License-Identifier: Apache-2.0 +use std::any::Any; use std::collections::HashMap; use std::sync::Arc; -use std::any::Any; -use arrow::datatypes::{SchemaRef, SchemaBuilder, Field}; +use arrow::datatypes::{Field, SchemaBuilder, SchemaRef}; use datafusion::datasource::file_format::{parquet::ParquetFormat, FileFormat}; -use datafusion::datasource::physical_plan::{FileSinkConfig, FileScanConfig}; +use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; use datafusion::execution::context::SessionState; use datafusion::physical_expr::PhysicalSortRequirement; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::{ExecutionPlan, PhysicalExpr}; -use datafusion_common::{Result, Statistics, FileType, project_schema}; +use datafusion_common::{project_schema, FileType, Result, Statistics}; -use object_store::{ObjectStore, ObjectMeta}; +use object_store::{ObjectMeta, ObjectStore}; use async_trait::async_trait; @@ -39,7 +39,7 @@ pub struct LakeSoulParquetFormat { impl LakeSoulParquetFormat { pub fn new(parquet_format: Arc, conf: LakeSoulIOConfig) -> Self { - Self { parquet_format, conf} + Self { parquet_format, conf } } pub fn primary_keys(&self) -> Arc> { @@ -57,7 +57,7 @@ impl LakeSoulParquetFormat { #[async_trait] impl FileFormat for LakeSoulParquetFormat { - fn as_any(&self) -> &dyn Any { + fn as_any(&self) -> &dyn Any { self } @@ -77,7 +77,9 @@ impl FileFormat for LakeSoulParquetFormat { table_schema: SchemaRef, object: &ObjectMeta, ) -> Result { - self.parquet_format.infer_stats(state, store, table_schema, object).await + self.parquet_format + .infer_stats(state, store, table_schema, object) + .await } async fn create_physical_plan( @@ -94,18 +96,14 @@ impl FileFormat for LakeSoulParquetFormat { .enable_pruning(state.config_options()) .then(|| filters.cloned()) .flatten(); - + let file_schema = conf.file_schema.clone(); let mut builder = SchemaBuilder::from(file_schema.fields()); for field in &conf.table_partition_cols { builder.push(Field::new(field.name(), field.data_type().clone(), false)); } - let (summary_conf, flatten_conf) = flatten_file_scan_config( - state, - self.parquet_format.clone(), - conf, - self.conf.primary_keys_slice(), - ).await?; + let (summary_conf, flatten_conf) = + flatten_file_scan_config(state, self.parquet_format.clone(), conf, self.conf.primary_keys_slice()).await?; let projection = summary_conf.projection.clone(); let merge_schema = Arc::new(builder.finish()); @@ -115,24 +113,20 @@ impl FileFormat for LakeSoulParquetFormat { flatten_conf, predicate, self.parquet_format.metadata_size_hint(state.config_options()), - self.conf.clone() + self.conf.clone(), )); if let Some(projection) = projection { let mut projection_expr = vec![]; for idx in projection { projection_expr.push(( - datafusion::physical_expr::expressions::col( - merge_schema.field(idx).name(), - &merge_schema, - )?, - merge_schema.field(idx).name().clone() + datafusion::physical_expr::expressions::col(merge_schema.field(idx).name(), &merge_schema)?, + merge_schema.field(idx).name().clone(), )); } Ok(Arc::new(ProjectionExec::try_new(projection_expr, merge_exec)?)) } else { Ok(merge_exec) } - } async fn create_writer_physical_plan( @@ -142,13 +136,14 @@ impl FileFormat for LakeSoulParquetFormat { conf: FileSinkConfig, order_requirements: Option>, ) -> Result> { - self.parquet_format.create_writer_physical_plan(input, state, conf, order_requirements).await + self.parquet_format + .create_writer_physical_plan(input, state, conf, order_requirements) + .await } fn file_type(&self) -> FileType { FileType::PARQUET } - } async fn flatten_file_scan_config( @@ -169,43 +164,49 @@ async fn flatten_file_scan_config( let objects = &[file.object_meta.clone()]; let file_groups = vec![vec![file.clone()]]; let file_schema = format.infer_schema(state, &store, objects).await?; - let statistics = format.infer_stats(state, &store, file_schema.clone(), &file.object_meta).await?; - let projection = compute_project_column_indices(file_schema.clone(), projected_schema.clone(), primary_keys); + let statistics = format + .infer_stats(state, &store, file_schema.clone(), &file.object_meta) + .await?; + let projection = + compute_project_column_indices(file_schema.clone(), projected_schema.clone(), primary_keys); let limit = conf.limit; let table_partition_cols = vec![]; let output_ordering = conf.output_ordering.clone(); let infinite_source = conf.infinite_source; - let config = FileScanConfig { - object_store_url: object_store_url.clone(), - file_schema, - file_groups, - statistics, - projection, - limit, - table_partition_cols, - output_ordering, - infinite_source + let config = FileScanConfig { + object_store_url: object_store_url.clone(), + file_schema, + file_groups, + statistics, + projection, + limit, + table_partition_cols, + output_ordering, + infinite_source, }; flatten_configs.push(config); } } Ok((summary_conf, flatten_configs)) - - } -fn compute_project_column_indices(schema: SchemaRef, project_schema: SchemaRef, primary_keys: &[String]) -> Option> { - Some(schema - .fields() - .iter() - .enumerate() - .filter_map(|(idx, field)| - if project_schema.field_with_name(field.name()).is_ok() | primary_keys.contains(field.name()) { - Some(idx) - } else { - None - } - ) - .collect::>() +fn compute_project_column_indices( + schema: SchemaRef, + project_schema: SchemaRef, + primary_keys: &[String], +) -> Option> { + Some( + schema + .fields() + .iter() + .enumerate() + .filter_map(|(idx, field)| { + if project_schema.field_with_name(field.name()).is_ok() | primary_keys.contains(field.name()) { + Some(idx) + } else { + None + } + }) + .collect::>(), ) -} \ No newline at end of file +} diff --git a/rust/lakesoul-io/src/datasource/listing.rs b/rust/lakesoul-io/src/datasource/listing.rs index e746104b9..1d10d654b 100644 --- a/rust/lakesoul-io/src/datasource/listing.rs +++ b/rust/lakesoul-io/src/datasource/listing.rs @@ -2,22 +2,22 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::sync::Arc; use std::any::Any; +use std::sync::Arc; use async_trait::async_trait; -use arrow::datatypes::{SchemaRef, SchemaBuilder}; +use arrow::datatypes::{SchemaBuilder, SchemaRef}; use datafusion::datasource::file_format::FileFormat; +use datafusion::datasource::listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}; use datafusion::datasource::physical_plan::FileSinkConfig; use datafusion::execution::context::SessionState; use datafusion::physical_plan::ExecutionPlan; use datafusion::{datasource::TableProvider, logical_expr::Expr}; -use datafusion::datasource::listing::{ListingTable, ListingTableUrl, ListingOptions, ListingTableConfig}; -use datafusion::logical_expr::{TableType, TableProviderFilterPushDown}; -use datafusion_common::{Result, FileTypeWriterOptions}; +use datafusion::logical_expr::{TableProviderFilterPushDown, TableType}; +use datafusion_common::{FileTypeWriterOptions, Result}; use crate::lakesoul_io_config::LakeSoulIOConfig; use crate::transform::uniform_schema; @@ -28,43 +28,50 @@ pub struct LakeSoulListingTable { lakesoul_io_config: LakeSoulIOConfig, } - impl LakeSoulListingTable { pub fn new(listing_table: Arc, lakesoul_io_config: LakeSoulIOConfig) -> Self { - Self { listing_table, lakesoul_io_config } + Self { + listing_table, + lakesoul_io_config, + } } - pub async fn new_with_config_and_format(session_state: &SessionState, lakesoul_io_config: LakeSoulIOConfig, file_format: Arc, as_sink: bool) -> Result { + pub async fn new_with_config_and_format( + session_state: &SessionState, + lakesoul_io_config: LakeSoulIOConfig, + file_format: Arc, + as_sink: bool, + ) -> Result { let config = match as_sink { false => { // Parse the path - let table_paths = lakesoul_io_config.files.iter().map(ListingTableUrl::parse).collect::>>()?; + let table_paths = lakesoul_io_config + .files + .iter() + .map(ListingTableUrl::parse) + .collect::>>()?; // Create default parquet options - let object_store_url = table_paths.get(0).unwrap().object_store(); + let object_store_url = table_paths.first().unwrap().object_store(); let store = session_state.runtime_env().object_store(object_store_url.clone())?; let target_schema = uniform_schema(lakesoul_io_config.schema()); - - let listing_options = ListingOptions::new(file_format.clone()) - .with_file_extension(".parquet"); + + let listing_options = ListingOptions::new(file_format.clone()).with_file_extension(".parquet"); // .with_table_partition_cols(table_partition_cols); - let mut objects = vec![]; + let mut objects = vec![]; for url in &table_paths { objects.push(store.head(url.prefix()).await?); } // Resolve the schema - let resolved_schema = file_format - .infer_schema(session_state, &store, &objects) - .await?; - - + let resolved_schema = file_format.infer_schema(session_state, &store, &objects).await?; + let mut builder = SchemaBuilder::from(target_schema.fields()); for field in resolved_schema.fields() { if target_schema.field_with_name(field.name()).is_err() { builder.push(field.clone()); } } - + ListingTableConfig::new_with_multi_paths(table_paths) .with_listing_options(listing_options) .with_schema(Arc::new(builder.finish())) @@ -74,32 +81,28 @@ impl LakeSoulListingTable { let table_partition_cols = lakesoul_io_config .range_partitions .iter() - .map(|col| - Ok(( - col.clone(), - target_schema.field_with_name(col)?.data_type().clone() - )) - ) + .map(|col| Ok((col.clone(), target_schema.field_with_name(col)?.data_type().clone()))) .collect::>>()?; let listing_options = ListingOptions::new(file_format.clone()) .with_file_extension(".parquet") .with_table_partition_cols(table_partition_cols) .with_insert_mode(datafusion::datasource::listing::ListingTableInsertMode::AppendNewFiles); - let prefix = ListingTableUrl::parse_create_local_if_not_exists(lakesoul_io_config.prefix.clone(), true)?; - + let prefix = + ListingTableUrl::parse_create_local_if_not_exists(lakesoul_io_config.prefix.clone(), true)?; + ListingTableConfig::new(prefix) .with_listing_options(listing_options) .with_schema(target_schema) } }; - + // Create a new TableProvider let listing_table = Arc::new(ListingTable::try_new(config)?); - Ok(Self { - listing_table, - lakesoul_io_config + Ok(Self { + listing_table, + lakesoul_io_config, }) } @@ -138,7 +141,11 @@ impl TableProvider for LakeSoulListingTable { .iter() .map(|f| { if let Ok(cols) = f.to_columns() { - if self.lakesoul_io_config.parquet_filter_pushdown && cols.iter().all(|col| self.lakesoul_io_config.primary_keys.contains(&col.name)) { + if self.lakesoul_io_config.parquet_filter_pushdown + && cols + .iter() + .all(|col| self.lakesoul_io_config.primary_keys.contains(&col.name)) + { Ok(TableProviderFilterPushDown::Inexact) } else { Ok(TableProviderFilterPushDown::Unsupported) @@ -161,7 +168,7 @@ impl TableProvider for LakeSoulListingTable { ) -> Result> { self.listing_table.scan(state, projection, filters, limit).await } - + async fn insert_into( &self, state: &SessionState, @@ -223,10 +230,7 @@ impl TableProvider for LakeSoulListingTable { let file_type_writer_options = match &self.options().file_type_write_options { Some(opt) => opt.clone(), - None => FileTypeWriterOptions::build_default( - &file_format.file_type(), - state.config_options(), - )?, + None => FileTypeWriterOptions::build_default(&file_format.file_type(), state.config_options())?, }; // Sink related option, apart from format @@ -282,6 +286,6 @@ impl TableProvider for LakeSoulListingTable { self.options() .format .create_writer_physical_plan(input, state, config, order_requirements) - .await + .await } } diff --git a/rust/lakesoul-io/src/datasource/mod.rs b/rust/lakesoul-io/src/datasource/mod.rs index 8c87d8f51..fb202adea 100644 --- a/rust/lakesoul-io/src/datasource/mod.rs +++ b/rust/lakesoul-io/src/datasource/mod.rs @@ -2,8 +2,8 @@ // // SPDX-License-Identifier: Apache-2.0 -pub mod parquet_source; pub mod empty_schema; pub mod file_format; pub mod listing; -mod physical_plan; \ No newline at end of file +pub mod parquet_source; +mod physical_plan; diff --git a/rust/lakesoul-io/src/datasource/parquet_source.rs b/rust/lakesoul-io/src/datasource/parquet_source.rs index 450b2d8cb..13293fae7 100644 --- a/rust/lakesoul-io/src/datasource/parquet_source.rs +++ b/rust/lakesoul-io/src/datasource/parquet_source.rs @@ -62,14 +62,14 @@ impl LakeSoulParquetProvider { for i in 0..self.config.files.len() { let file = self.config.files[i].clone(); let df = context.read_parquet(file, Default::default()).await.unwrap(); - full_schema.merge(&Schema::try_from(df.schema()).unwrap().to_dfschema().unwrap()); + full_schema.merge(&Schema::from(df.schema()).to_dfschema().unwrap()); let plan = df.into_unoptimized_plan(); plans.push(plan); } Ok(Self { config: self.config.clone(), plans, - full_schema: SchemaRef::new(Schema::try_from(full_schema).unwrap()), + full_schema: SchemaRef::new(Schema::from(full_schema)), }) } @@ -208,7 +208,7 @@ impl LakeSoulParquetScanExec { merge_operators: Arc>, primary_keys: Arc>, ) -> Self { - let target_schema_with_pks = if let Some(proj) = projections { + let target_schema_with_pks = if let Some(proj) = projections { let mut proj_with_pks = proj.clone(); for idx in 0..primary_keys.len() { let field_idx = full_schema.index_of(primary_keys[idx].as_str()).unwrap(); @@ -220,7 +220,7 @@ impl LakeSoulParquetScanExec { } else { full_schema.clone() }; - + Self { projections: projections.unwrap().clone(), origin_schema: full_schema.clone(), @@ -290,11 +290,8 @@ impl ExecutionPlan for LakeSoulParquetScanExec { .projections .iter() .map(|&idx| { - datafusion::physical_expr::expressions::col( - self.origin_schema().field(idx).name(), - &self.schema(), - ) - .unwrap() + datafusion::physical_expr::expressions::col(self.origin_schema().field(idx).name(), &self.schema()) + .unwrap() }) .collect::>(), schema: self.target_schema.clone(), @@ -303,7 +300,6 @@ impl ExecutionPlan for LakeSoulParquetScanExec { Ok(Box::pin(result)) } - } pub fn merge_stream( @@ -364,7 +360,10 @@ pub fn merge_stream( } fn schema_intersection(df_schema: DFSchemaRef, request_schema: SchemaRef, primary_keys: &[String]) -> Vec { - let mut exprs = primary_keys.iter().map(|pk| Column(datafusion::common::Column::new_unqualified(pk))).collect::>(); + let mut exprs = primary_keys + .iter() + .map(|pk| Column(datafusion::common::Column::new_unqualified(pk))) + .collect::>(); for field in request_schema.fields() { if primary_keys.contains(field.name()) { continue; diff --git a/rust/lakesoul-io/src/datasource/physical_plan/empty_schema.rs b/rust/lakesoul-io/src/datasource/physical_plan/empty_schema.rs index ad8263eca..18d42d3ad 100644 --- a/rust/lakesoul-io/src/datasource/physical_plan/empty_schema.rs +++ b/rust/lakesoul-io/src/datasource/physical_plan/empty_schema.rs @@ -5,8 +5,12 @@ use std::any::Any; use std::sync::Arc; -use arrow_schema::{SchemaRef, Schema}; -use datafusion::{physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream}, physical_expr::PhysicalSortExpr, execution::TaskContext}; +use arrow_schema::{Schema, SchemaRef}; +use datafusion::{ + execution::TaskContext, + physical_expr::PhysicalSortExpr, + physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream}, +}; use datafusion_common::Result; use crate::default_column_stream::empty_schema_stream::EmptySchemaStream; @@ -63,6 +67,4 @@ impl ExecutionPlan for EmptySchemaScanExec { self.count, ))) } - } - diff --git a/rust/lakesoul-io/src/datasource/physical_plan/merge.rs b/rust/lakesoul-io/src/datasource/physical_plan/merge.rs index c40f0d939..966e98e04 100644 --- a/rust/lakesoul-io/src/datasource/physical_plan/merge.rs +++ b/rust/lakesoul-io/src/datasource/physical_plan/merge.rs @@ -2,12 +2,17 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::{any::Any, collections::HashMap}; use std::sync::Arc; +use std::{any::Any, collections::HashMap}; -use arrow_schema::{SchemaRef, Field, Schema}; -use datafusion::{physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, PhysicalExpr}, physical_expr::PhysicalSortExpr, execution::TaskContext, datasource::physical_plan::{FileScanConfig, ParquetExec}}; -use datafusion_common::{Result, DataFusionError}; +use arrow_schema::{Field, Schema, SchemaRef}; +use datafusion::{ + datasource::physical_plan::{FileScanConfig, ParquetExec}, + execution::TaskContext, + physical_expr::PhysicalSortExpr, + physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PhysicalExpr, SendableRecordBatchStream}, +}; +use datafusion_common::{DataFusionError, Result}; use crate::datasource::parquet_source::merge_stream; use crate::lakesoul_io_config::LakeSoulIOConfig; @@ -19,7 +24,7 @@ pub struct MergeParquetExec { default_column_value: Arc>, merge_operators: Arc>, config: FileScanConfig, - inputs: Vec> + inputs: Vec>, } impl MergeParquetExec { @@ -61,8 +66,8 @@ impl MergeParquetExec { let primary_keys = Arc::new(io_config.primary_keys); let default_column_value = Arc::new(io_config.default_column_value); let merge_operators = Arc::new(io_config.merge_operators); - - Self { + + Self { schema, inputs, config, @@ -113,26 +118,21 @@ impl ExecutionPlan for MergeParquetExec { } fn with_new_children(self: Arc, inputs: Vec>) -> Result> { - Ok(Arc::new( - Self { - schema: self.schema(), - inputs, - primary_keys: self.primary_keys(), - default_column_value: self.default_column_value(), - merge_operators: self.merge_operators(), - config: self.config.clone(), - } - )) + Ok(Arc::new(Self { + schema: self.schema(), + inputs, + primary_keys: self.primary_keys(), + default_column_value: self.default_column_value(), + merge_operators: self.merge_operators(), + config: self.config.clone(), + })) } - fn execute(&self, - partition: usize, - context: Arc - ) -> Result { + fn execute(&self, partition: usize, context: Arc) -> Result { if partition != 0 { - return Err(DataFusionError::Internal( - format!("Invalid requested partition {partition}. InsertExec requires a single input partition." - ))); + return Err(DataFusionError::Internal(format!( + "Invalid requested partition {partition}. InsertExec requires a single input partition." + ))); } let mut stream_init_futs = Vec::with_capacity(self.inputs.len()); @@ -159,8 +159,5 @@ impl ExecutionPlan for MergeParquetExec { )?; Ok(merged_stream) - } - } - diff --git a/rust/lakesoul-io/src/datasource/physical_plan/mod.rs b/rust/lakesoul-io/src/datasource/physical_plan/mod.rs index 0eb61cf13..f5d46b0b1 100644 --- a/rust/lakesoul-io/src/datasource/physical_plan/mod.rs +++ b/rust/lakesoul-io/src/datasource/physical_plan/mod.rs @@ -2,8 +2,8 @@ // // SPDX-License-Identifier: Apache-2.0 -pub use merge::MergeParquetExec; pub use empty_schema::EmptySchemaScanExec; +pub use merge::MergeParquetExec; +mod empty_schema; mod merge; -mod empty_schema; \ No newline at end of file diff --git a/rust/lakesoul-io/src/filter/parser.rs b/rust/lakesoul-io/src/filter/parser.rs index 316b2209f..b42a5b1ff 100644 --- a/rust/lakesoul-io/src/filter/parser.rs +++ b/rust/lakesoul-io/src/filter/parser.rs @@ -2,12 +2,12 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::ops::Not; use arrow_schema::{DataType, Field, SchemaRef}; use datafusion::logical_expr::Expr; use datafusion::prelude::col; use datafusion::scalar::ScalarValue; - +use std::ops::Not; + pub struct Parser {} impl Parser { diff --git a/rust/lakesoul-io/src/hdfs/mod.rs b/rust/lakesoul-io/src/hdfs/mod.rs index 9a7f98ebf..9bec9f6db 100644 --- a/rust/lakesoul-io/src/hdfs/mod.rs +++ b/rust/lakesoul-io/src/hdfs/mod.rs @@ -159,7 +159,7 @@ impl ObjectStore for Hdfs { // store: "hdfs", // source: Box::new(e), // }))), - // meta: + // meta: // }) // } diff --git a/rust/lakesoul-io/src/hdfs/util.rs b/rust/lakesoul-io/src/hdfs/util.rs index b9d784f8c..169888aa9 100644 --- a/rust/lakesoul-io/src/hdfs/util.rs +++ b/rust/lakesoul-io/src/hdfs/util.rs @@ -11,9 +11,9 @@ use futures::{stream::StreamExt, TryStreamExt}; #[cfg(not(target_arch = "wasm32"))] /// Takes a function and spawns it to a tokio blocking pool if available pub async fn maybe_spawn_blocking(f: F) -> object_store::Result - where - F: FnOnce() -> object_store::Result + Send + 'static, - T: Send + 'static, +where + F: FnOnce() -> object_store::Result + Send + 'static, + T: Send + 'static, { match tokio::runtime::Handle::try_current() { Ok(runtime) => runtime.spawn_blocking(f).await?, @@ -42,9 +42,9 @@ pub async fn coalesce_ranges( fetch: F, coalesce: usize, ) -> object_store::Result> - where - F: Send + FnMut(std::ops::Range) -> Fut, - Fut: std::future::Future> + Send, +where + F: Send + FnMut(std::ops::Range) -> Fut, + Fut: std::future::Future> + Send, { let fetch_ranges = merge_ranges(ranges, coalesce, OBJECT_STORE_COALESCE_MAX); @@ -91,10 +91,10 @@ fn merge_ranges( while end_idx != ranges.len() && ranges[end_idx] - .start - .checked_sub(range_end) - .map(|delta| delta <= coalesce) - .unwrap_or(true) + .start + .checked_sub(range_end) + .map(|delta| delta <= coalesce) + .unwrap_or(true) { range_end = range_end.max(ranges[end_idx].end); end_idx += 1; diff --git a/rust/lakesoul-io/src/helpers.rs b/rust/lakesoul-io/src/helpers.rs index a4fa39207..7a89c85ca 100644 --- a/rust/lakesoul-io/src/helpers.rs +++ b/rust/lakesoul-io/src/helpers.rs @@ -3,44 +3,51 @@ // SPDX-License-Identifier: Apache-2.0 use arrow_schema::Schema; -use datafusion::{execution::context::SessionState, physical_expr::{create_physical_expr, PhysicalSortExpr}, physical_planner::create_physical_sort_expr, logical_expr::col, physical_plan::Partitioning}; +use datafusion::{ + execution::context::SessionState, + logical_expr::col, + physical_expr::{create_physical_expr, PhysicalSortExpr}, + physical_plan::Partitioning, + physical_planner::create_physical_sort_expr, +}; use datafusion_common::{DFSchema, Result}; - pub fn create_sort_exprs( colunms: &[String], input_dfschema: &DFSchema, input_schema: &Schema, - session_state: &SessionState + session_state: &SessionState, ) -> Result> { colunms .iter() - .map(|column| + .map(|column| { create_physical_sort_expr( - &col(column).sort(true, true), - input_dfschema, - input_schema, - session_state.execution_props()) - ) + &col(column).sort(true, true), + input_dfschema, + input_schema, + session_state.execution_props(), + ) + }) .collect::>>() } pub fn create_hash_partitioning( - colunms: &[String], - partitioning_num: usize, + colunms: &[String], + partitioning_num: usize, input_dfschema: &DFSchema, input_schema: &Schema, - session_state: &SessionState + session_state: &SessionState, ) -> Result { let runtime_expr = colunms .iter() - .map(|column| + .map(|column| { create_physical_expr( - &col(column), - input_dfschema, - input_schema, - session_state.execution_props()) - ) + &col(column), + input_dfschema, + input_schema, + session_state.execution_props(), + ) + }) .collect::>>()?; Ok(Partitioning::Hash(runtime_expr, partitioning_num)) } diff --git a/rust/lakesoul-io/src/lakesoul_io_config.rs b/rust/lakesoul-io/src/lakesoul_io_config.rs index d2022ba95..e90a45ff9 100644 --- a/rust/lakesoul-io/src/lakesoul_io_config.rs +++ b/rust/lakesoul-io/src/lakesoul_io_config.rs @@ -10,12 +10,13 @@ use datafusion::execution::context::{QueryPlanner, SessionState}; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::logical_expr::Expr; use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_common::DataFusionError::ObjectStore; +use datafusion_common::DataFusionError::{External, ObjectStore}; use derivative::Derivative; use object_store::aws::AmazonS3Builder; -use object_store::RetryConfig; +use object_store::{ClientOptions, RetryConfig}; use std::collections::HashMap; use std::sync::Arc; +use std::time::Duration; use url::{ParseError, Url}; #[cfg(feature = "hdfs")] @@ -209,8 +210,8 @@ impl LakeSoulIOConfigBuilder { self } - pub fn with_object_store_option(mut self, key: String, value: String) -> Self { - self.config.object_store_options.insert(key, value); + pub fn with_object_store_option(mut self, key: impl Into, value: impl Into) -> Self { + self.config.object_store_options.insert(key.into(), value.into()); self } @@ -237,9 +238,9 @@ impl LakeSoulIOConfigBuilder { } impl From for LakeSoulIOConfigBuilder { - fn from(val: LakeSoulIOConfig) -> Self { - LakeSoulIOConfigBuilder { config: val } - } + fn from(val: LakeSoulIOConfig) -> Self { + LakeSoulIOConfigBuilder { config: val } + } } /// First check envs for credentials, region and endpoint. @@ -259,10 +260,32 @@ pub fn register_s3_object_store(url: &Url, config: &LakeSoulIOConfig, runtime: & .ok() .or_else(|| config.object_store_options.get("fs.s3a.endpoint.region").cloned()) }); - let endpoint = std::env::var("AWS_ENDPOINT") + let mut endpoint = std::env::var("AWS_ENDPOINT") .ok() .or_else(|| config.object_store_options.get("fs.s3a.endpoint").cloned()); let bucket = config.object_store_options.get("fs.s3a.bucket").cloned(); + let virtual_path_style = config.object_store_options.get("fs.s3a.path.style.access").cloned(); + let virtual_path_style = virtual_path_style.is_some_and(|s| s == "true"); + if !virtual_path_style { + if let (Some(endpoint_str), Some(bucket)) = (&endpoint, &bucket) { + // for host style access with endpoint defined, we need to check endpoint contains bucket name + if !endpoint_str.contains(bucket) { + let mut endpoint_url = Url::parse(endpoint_str.as_str()).map_err(|e| External(Box::new(e)))?; + endpoint_url + .set_host(Some(&*format!( + "{}.{}", + bucket, + endpoint_url.host_str().expect("endpoint should contains host") + ))) + .map_err(|e| External(Box::new(e)))?; + let endpoint_s = endpoint_url.to_string(); + endpoint = endpoint_s + .strip_suffix('/') + .map(|s| s.to_string()) + .or(Some(endpoint_s)); + } + } + } if bucket.is_none() { return Err(DataFusionError::ArrowError(ArrowError::InvalidArgumentError( @@ -275,6 +298,14 @@ pub fn register_s3_object_store(url: &Url, config: &LakeSoulIOConfig, runtime: & .with_region(region.unwrap_or_else(|| "us-east-1".to_owned())) .with_bucket_name(bucket.unwrap()) .with_retry(retry_config) + .with_virtual_hosted_style_request(!virtual_path_style) + .with_client_options( + ClientOptions::new() + .with_allow_http(true) + .with_connect_timeout(Duration::from_secs(10)) + .with_pool_idle_timeout(Duration::from_secs(300)) + .with_timeout(Duration::from_secs(10)), + ) .with_allow_http(true); if let (Some(k), Some(s)) = (key, secret) { s3_store_builder = s3_store_builder.with_access_key_id(k).with_secret_access_key(s); @@ -376,13 +407,14 @@ pub fn create_session_context(config: &mut LakeSoulIOConfig) -> Result>) -> Result { +pub fn create_session_context_with_planner( + config: &mut LakeSoulIOConfig, + planner: Option>, +) -> Result { let mut sess_conf = SessionConfig::default() .with_batch_size(config.batch_size) .with_parquet_pruning(true) - // .with_repartition_sorts(false) - // .with_prefetch(config.prefetch_size) - ; + .with_prefetch(config.prefetch_size); sess_conf.options_mut().optimizer.enable_round_robin_repartition = false; // if true, the record_batches poll from stream become unordered sess_conf.options_mut().optimizer.prefer_hash_join = false; //if true, panicked at 'range end out of bounds' @@ -414,12 +446,11 @@ pub fn create_session_context_with_planner(config: &mut LakeSoulIOConfig, planne // create session context let state = if let Some(planner) = planner { - SessionState::new_with_config_rt(sess_conf, Arc::new(runtime)) - .with_query_planner(planner) + SessionState::new_with_config_rt(sess_conf, Arc::new(runtime)).with_query_planner(planner) } else { SessionState::new_with_config_rt(sess_conf, Arc::new(runtime)) }; - + Ok(SessionContext::new_with_state(state)) } diff --git a/rust/lakesoul-io/src/lakesoul_reader.rs b/rust/lakesoul-io/src/lakesoul_reader.rs index 036c922d4..3402d8681 100644 --- a/rust/lakesoul-io/src/lakesoul_reader.rs +++ b/rust/lakesoul-io/src/lakesoul_reader.rs @@ -55,8 +55,17 @@ impl LakeSoulReader { } else { // let source = LakeSoulParquetProvider::from_config(self.config.clone()); // let source = source.build_with_context(&self.sess_ctx).await.unwrap(); - let file_format = Arc::new(LakeSoulParquetFormat::new(Arc::new(ParquetFormat::new()), self.config.clone())); - let source = LakeSoulListingTable::new_with_config_and_format(&self.sess_ctx.state(), self.config.clone(), file_format, false).await?; + let file_format = Arc::new(LakeSoulParquetFormat::new( + Arc::new(ParquetFormat::new()), + self.config.clone(), + )); + let source = LakeSoulListingTable::new_with_config_and_format( + &self.sess_ctx.state(), + self.config.clone(), + file_format, + false, + ) + .await?; let dataframe = self.sess_ctx.read_table(Arc::new(source))?; let stream = prune_filter_and_execute( diff --git a/rust/lakesoul-io/src/lakesoul_writer.rs b/rust/lakesoul-io/src/lakesoul_writer.rs index 3d8c301f2..c6c0a48de 100644 --- a/rust/lakesoul-io/src/lakesoul_writer.rs +++ b/rust/lakesoul-io/src/lakesoul_writer.rs @@ -18,7 +18,7 @@ use datafusion::physical_expr::{PhysicalExpr, PhysicalSortExpr}; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchReceiverStreamBuilder}; -use datafusion::physical_plan::{ExecutionPlan, Partitioning, SendableRecordBatchStream, DisplayAs, DisplayFormatType}; +use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream}; use datafusion_common::DataFusionError; use datafusion_common::DataFusionError::Internal; use object_store::path::Path; @@ -86,10 +86,7 @@ struct InMemBuf(Arc>>); impl Write for InMemBuf { #[inline] fn write(&mut self, buf: &[u8]) -> std::io::Result { - let mut v = self - .0 - .try_borrow_mut() - .map_err(|_| std::io::Error::from(AddrInUse))?; + let mut v = self.0.try_borrow_mut().map_err(|_| std::io::Error::from(AddrInUse))?; v.extend(buf); Ok(buf.len()) } @@ -101,10 +98,7 @@ impl Write for InMemBuf { #[inline] fn write_all(&mut self, buf: &[u8]) -> std::io::Result<()> { - let mut v = self - .0 - .try_borrow_mut() - .map_err(|_| std::io::Error::from(AddrInUse))?; + let mut v = self.0.try_borrow_mut().map_err(|_| std::io::Error::from(AddrInUse))?; v.extend(buf); Ok(()) } @@ -165,11 +159,10 @@ impl ExecutionPlan for ReceiverStreamExec { let builder = self.receiver_stream_builder.borrow_mut().take().unwrap(); Ok(builder.build()) } - } impl MultiPartAsyncWriter { - pub async fn try_new_with_context(config:&mut LakeSoulIOConfig, task_context: Arc) -> Result { + pub async fn try_new_with_context(config: &mut LakeSoulIOConfig, task_context: Arc) -> Result { if config.files.is_empty() { return Err(Internal("wrong number of file names provided for writer".to_string())); } @@ -215,7 +208,7 @@ impl MultiPartAsyncWriter { _config: config.clone(), object_store, path, - num_rows: 0 + num_rows: 0, }) } @@ -261,7 +254,6 @@ impl MultiPartAsyncWriter { pub fn task_ctx(&self) -> Arc { self.task_context.clone() } - } #[async_trait] @@ -501,11 +493,11 @@ mod tests { }; use arrow::array::{ArrayRef, Int64Array}; use arrow::record_batch::RecordBatch; + use arrow_array::Array; use datafusion::error::Result; use parquet::arrow::arrow_reader::ParquetRecordBatchReader; use std::fs::File; use std::sync::Arc; - use arrow_array::Array; use tokio::runtime::Builder; #[test] diff --git a/rust/lakesoul-io/src/lib.rs b/rust/lakesoul-io/src/lib.rs index 090c11512..4b95dea03 100644 --- a/rust/lakesoul-io/src/lib.rs +++ b/rust/lakesoul-io/src/lib.rs @@ -2,26 +2,25 @@ // // SPDX-License-Identifier: Apache-2.0 - -pub mod lakesoul_reader; -pub mod filter; -pub mod lakesoul_writer; -pub mod lakesoul_io_config; -pub mod sorted_merge; pub mod datasource; -pub mod repartition; +pub mod filter; pub mod helpers; +pub mod lakesoul_io_config; +pub mod lakesoul_reader; +pub mod lakesoul_writer; mod projection; +pub mod repartition; +pub mod sorted_merge; #[cfg(feature = "hdfs")] mod hdfs; -mod default_column_stream; mod constant; +mod default_column_stream; mod transform; +pub use arrow; +pub use datafusion; pub use datafusion::arrow::error::Result; +pub use serde_json; pub use tokio; -pub use datafusion; -pub use arrow; -pub use serde_json; \ No newline at end of file diff --git a/rust/lakesoul-io/src/projection/mod.rs b/rust/lakesoul-io/src/projection/mod.rs index 100722239..833f85a96 100644 --- a/rust/lakesoul-io/src/projection/mod.rs +++ b/rust/lakesoul-io/src/projection/mod.rs @@ -22,22 +22,16 @@ impl ProjectionStream { let arrays = self .expr .iter() - .map(|expr| { - expr.evaluate(batch) - .and_then(|v| v.into_array(batch.num_rows())) - }) + .map(|expr| expr.evaluate(batch).and_then(|v| v.into_array(batch.num_rows()))) .collect::>>()?; if arrays.is_empty() { - let options = - RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); - RecordBatch::try_new_with_options(self.schema.clone(), arrays, &options) - .map_err(Into::into) + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + RecordBatch::try_new_with_options(self.schema.clone(), arrays, &options).map_err(Into::into) } else { RecordBatch::try_new(self.schema.clone(), arrays).map_err(Into::into) } } - } /// Projection iterator diff --git a/rust/lakesoul-io/src/repartition/distributor_channels.rs b/rust/lakesoul-io/src/repartition/distributor_channels.rs index e71b88467..bba0e9fe5 100644 --- a/rust/lakesoul-io/src/repartition/distributor_channels.rs +++ b/rust/lakesoul-io/src/repartition/distributor_channels.rs @@ -48,9 +48,7 @@ use std::{ use parking_lot::Mutex; /// Create `n` empty channels. -pub fn channels( - n: usize, -) -> (Vec>, Vec>) { +pub fn channels(n: usize) -> (Vec>, Vec>) { let channels = (0..n) .map(|id| { Arc::new(Mutex::new(Channel { @@ -192,9 +190,7 @@ impl<'a, T> Future for SendFuture<'a, T> { // receiver end still alive? if !guard_channel.recv_alive { - return Poll::Ready(Err(SendError( - this.element.take().expect("just checked"), - ))); + return Poll::Ready(Err(SendError(this.element.take().expect("just checked")))); } let mut guard_gate = this.gate.lock(); @@ -202,16 +198,12 @@ impl<'a, T> Future for SendFuture<'a, T> { // does ANY receiver need data? // if so, allow sender to create another if guard_gate.empty_channels == 0 { - guard_gate - .send_wakers - .push((cx.waker().clone(), guard_channel.id)); + guard_gate.send_wakers.push((cx.waker().clone(), guard_channel.id)); return Poll::Pending; } let was_empty = guard_channel.data.is_empty(); - guard_channel - .data - .push_back(this.element.take().expect("just checked")); + guard_channel.data.push_back(this.element.take().expect("just checked")); if was_empty { guard_gate.empty_channels -= 1; guard_channel.wake_receivers(); @@ -373,10 +365,7 @@ impl Gate { /// This is helpful to signal that the receiver side is gone and the senders shall now error. fn wake_channel_senders(&mut self, id: usize) { // `drain_filter` is unstable, so implement our own - let (wake, keep) = self - .send_wakers - .drain(..) - .partition(|(_waker, id2)| id == *id2); + let (wake, keep) = self.send_wakers.drain(..).partition(|(_waker, id2)| id == *id2); self.send_wakers = keep; for (waker, _id) in wake { waker.wake(); diff --git a/rust/lakesoul-io/src/repartition/mod.rs b/rust/lakesoul-io/src/repartition/mod.rs index b452ac669..824e317c0 100644 --- a/rust/lakesoul-io/src/repartition/mod.rs +++ b/rust/lakesoul-io/src/repartition/mod.rs @@ -2,20 +2,39 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::{collections::HashMap, sync::Arc, any::Any, pin::Pin, task::{Context, Poll}}; +use std::{ + any::Any, + collections::HashMap, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; use arrow_schema::SchemaRef; -use datafusion::{physical_plan::{common::{AbortOnDropMany, AbortOnDropSingle, transpose}, PhysicalExpr, Partitioning, metrics::{ExecutionPlanMetricsSet, MetricBuilder, BaselineMetrics}, ExecutionPlan, DisplayFormatType, DisplayAs, SendableRecordBatchStream, sorts::streaming_merge::streaming_merge, RecordBatchStream}, execution::{memory_pool::{MemoryReservation, MemoryConsumer}, TaskContext}, physical_expr::PhysicalSortExpr}; -use datafusion_common::{DataFusionError, Result, hash_utils::create_hashes}; use datafusion::physical_plan::metrics; - -use arrow_array::{RecordBatch, builder::UInt64Builder, ArrayRef}; -use futures::{Stream, StreamExt, FutureExt}; +use datafusion::{ + execution::{ + memory_pool::{MemoryConsumer, MemoryReservation}, + TaskContext, + }, + physical_expr::PhysicalSortExpr, + physical_plan::{ + common::{transpose, AbortOnDropMany, AbortOnDropSingle}, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricBuilder}, + sorts::streaming_merge::streaming_merge, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, + SendableRecordBatchStream, + }, +}; +use datafusion_common::{hash_utils::create_hashes, DataFusionError, Result}; + +use arrow_array::{builder::UInt64Builder, ArrayRef, RecordBatch}; +use futures::{FutureExt, Stream, StreamExt}; use tokio::task::JoinHandle; use crate::repartition::distributor_channels::partition_aware_channels; -use self::distributor_channels::{DistributionSender, DistributionReceiver}; +use self::distributor_channels::{DistributionReceiver, DistributionSender}; use log::trace; use parking_lot::Mutex; @@ -29,7 +48,6 @@ type InputPartitionsToCurrentPartitionReceiver = Vec>; - /// Inner state of [`RepartitionExec`]. #[derive(Debug)] struct RepartitionByRangeAndHashExecState { @@ -77,7 +95,11 @@ impl BatchPartitioner { random_state: ahash::RandomState::with_seeds(0, 0, 0, 0), hash_buffer: vec![], }, - other => return Err(DataFusionError::NotImplemented(format!("Unsupported repartitioning scheme {other:?}"))), + other => { + return Err(DataFusionError::NotImplemented(format!( + "Unsupported repartitioning scheme {other:?}" + ))) + } }; Ok(Self { state, timer }) @@ -118,57 +140,51 @@ impl BatchPartitioner { num_partitions: partitions, hash_buffer, } = &mut self.state; - let it: Box> + Send> = - { - let timer = self.timer.timer(); + let it: Box> + Send> = { + let timer = self.timer.timer(); - let hash_arrays = hash_exprs - .iter() - .map(|expr| expr.evaluate(&batch)?.into_array(batch.num_rows())) - .collect::>>()?; + let hash_arrays = hash_exprs + .iter() + .map(|expr| expr.evaluate(&batch)?.into_array(batch.num_rows())) + .collect::>>()?; - hash_buffer.clear(); - hash_buffer.resize(batch.num_rows(), 0); + hash_buffer.clear(); + hash_buffer.resize(batch.num_rows(), 0); - create_hashes(&hash_arrays, random_state, hash_buffer)?; + create_hashes(&hash_arrays, random_state, hash_buffer)?; - let mut indices: Vec<_> = (0..*partitions) - .map(|_| UInt64Builder::with_capacity(batch.num_rows())) - .collect(); + let mut indices: Vec<_> = (0..*partitions) + .map(|_| UInt64Builder::with_capacity(batch.num_rows())) + .collect(); - for (index, hash) in hash_buffer.iter().enumerate() { - indices[(*hash % *partitions as u64) as usize] - .append_value(index as u64); - } + for (index, hash) in hash_buffer.iter().enumerate() { + indices[(*hash % *partitions as u64) as usize].append_value(index as u64); + } - let it = indices - .into_iter() - .enumerate() - .filter_map(|(partition, mut indices)| { - let indices = indices.finish(); - (!indices.is_empty()).then_some((partition, indices)) - }) - .map(move |(partition, indices)| { - // Produce batches based on indices - let columns = batch - .columns() - .iter() - .map(|c| { - arrow::compute::take(c.as_ref(), &indices, None) - .map_err(DataFusionError::ArrowError) - }) - .collect::>>()?; - - let batch = - RecordBatch::try_new(batch.schema(), columns).unwrap(); - - // bind timer so it drops w/ this iterator - let _ = &timer; - - Ok((partition, batch)) - }); - Box::new(it) - }; + let it = indices + .into_iter() + .enumerate() + .filter_map(|(partition, mut indices)| { + let indices = indices.finish(); + (!indices.is_empty()).then_some((partition, indices)) + }) + .map(move |(partition, indices)| { + // Produce batches based on indices + let columns = batch + .columns() + .iter() + .map(|c| arrow::compute::take(c.as_ref(), &indices, None).map_err(DataFusionError::ArrowError)) + .collect::>>()?; + + let batch = RecordBatch::try_new(batch.schema(), columns).unwrap(); + + // bind timer so it drops w/ this iterator + let _ = &timer; + + Ok((partition, batch)) + }); + Box::new(it) + }; Ok(it) } @@ -190,11 +206,7 @@ struct RepartitionMetrics { } impl RepartitionMetrics { - pub fn new( - output_partition: usize, - input_partition: usize, - metrics: &ExecutionPlanMetricsSet, - ) -> Self { + pub fn new(output_partition: usize, input_partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self { let label = metrics::Label::new("inputPartition", input_partition.to_string()); // Time in nanos to execute child operator and fetch batches @@ -233,10 +245,8 @@ pub struct RepartitionByRangeAndHashExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, - } - impl RepartitionByRangeAndHashExec { /// Input execution plan pub fn input(&self) -> &Arc { @@ -255,11 +265,7 @@ impl RepartitionByRangeAndHashExec { } impl DisplayAs for RepartitionByRangeAndHashExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!( @@ -271,11 +277,7 @@ impl DisplayAs for RepartitionByRangeAndHashExec { )?; if let Some(sort_exprs) = self.sort_exprs() { - write!( - f, - ", sort_exprs={}", - PhysicalSortExpr::format_list(sort_exprs) - )?; + write!(f, ", sort_exprs={}", PhysicalSortExpr::format_list(sort_exprs))?; } Ok(()) } @@ -283,15 +285,11 @@ impl DisplayAs for RepartitionByRangeAndHashExec { } } - impl RepartitionByRangeAndHashExec { /// Create a new RepartitionExec, that produces output `partitioning`, and /// does not preserve the order of the input (see [`Self::with_preserve_order`] /// for more details) - pub fn try_new( - input: Arc, - hash_partitioning: Partitioning, - ) -> Result { + pub fn try_new(input: Arc, hash_partitioning: Partitioning) -> Result { Ok(Self { input, hash_partitioning, @@ -315,16 +313,12 @@ impl RepartitionByRangeAndHashExec { async fn pull_from_input( input: Arc, partition: usize, - mut output_channels: HashMap< - usize, - (DistributionSender, SharedMemoryReservation), - >, + mut output_channels: HashMap, SharedMemoryReservation)>, partitioning: Partitioning, metrics: RepartitionMetrics, context: Arc, ) -> Result<()> { - let mut partitioner = - BatchPartitioner::try_new(partitioning, metrics.repartition_time.clone())?; + let mut partitioner = BatchPartitioner::try_new(partitioning, metrics.repartition_time.clone())?; // execute the child operator let timer = metrics.fetch_time.timer(); @@ -450,13 +444,10 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { vec![self.input.clone()] } - fn with_new_children( - self: Arc, - mut children: Vec>, - ) -> Result> { + fn with_new_children(self: Arc, mut children: Vec>) -> Result> { let repartition = RepartitionByRangeAndHashExec::try_new(children.swap_remove(0), self.hash_partitioning.clone())?; - + Ok(Arc::new(repartition)) } @@ -484,16 +475,8 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { vec![self.input().output_partitioning().partition_count() <= 1] } - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { - trace!( - "Start {}::execute for partition: {}", - self.name(), - partition - ); + fn execute(&self, partition: usize, context: Arc) -> Result { + trace!("Start {}::execute for partition: {}", self.name(), partition); // lock mutexes let mut state = self.state.lock(); @@ -502,30 +485,28 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { // if this is the first partition to be invoked then we need to set up initial state if state.channels.is_empty() { - let (txs, rxs) = { - let (txs, rxs) = - partition_aware_channels(num_input_partitions, num_output_partitions); + let (txs, rxs) = { + let (txs, rxs) = partition_aware_channels(num_input_partitions, num_output_partitions); // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition let txs = transpose(txs); let rxs = transpose(rxs); (txs, rxs) - // } else { - // // create one channel per *output* partition - // // note we use a custom channel that ensures there is always data for each receiver - // // but limits the amount of buffering if required. - // let (txs, rxs) = channels(num_output_partitions); - // // Clone sender for each input partitions - // let txs = txs - // .into_iter() - // .map(|item| vec![item; num_input_partitions]) - // .collect::>(); - // let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); - // (txs, rxs) + // } else { + // // create one channel per *output* partition + // // note we use a custom channel that ensures there is always data for each receiver + // // but limits the amount of buffering if required. + // let (txs, rxs) = channels(num_output_partitions); + // // Clone sender for each input partitions + // let txs = txs + // .into_iter() + // .map(|item| vec![item; num_input_partitions]) + // .collect::>(); + // let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); + // (txs, rxs) }; for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { let reservation = Arc::new(Mutex::new( - MemoryConsumer::new(format!("{}[{partition}]", self.name())) - .register(context.memory_pool()), + MemoryConsumer::new(format!("{}[{partition}]", self.name())).register(context.memory_pool()), )); state.channels.insert(partition, (tx, rx, reservation)); } @@ -536,22 +517,19 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { let txs: HashMap<_, _> = state .channels .iter() - .map(|(partition, (tx, _rx, reservation))| { - (*partition, (tx[i].clone(), Arc::clone(reservation))) - }) + .map(|(partition, (tx, _rx, reservation))| (*partition, (tx[i].clone(), Arc::clone(reservation)))) .collect(); let r_metrics = RepartitionMetrics::new(i, partition, &self.metrics); - let input_task: JoinHandle> = - tokio::spawn(Self::pull_from_input( - self.input.clone(), - i, - txs.clone(), - self.hash_partitioning().clone(), - r_metrics, - context.clone(), - )); + let input_task: JoinHandle> = tokio::spawn(Self::pull_from_input( + self.input.clone(), + i, + txs.clone(), + self.hash_partitioning().clone(), + r_metrics, + context.clone(), + )); // In a separate task, wait for each input to be done // (and pass along any errors, including panic!s) @@ -573,12 +551,9 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { partition ); - // now return stream for the specified *output* partition which will + // now return stream for the specified *output* partition which will // read from the channel - let (_tx, rx, reservation) = state - .channels - .remove(&partition) - .expect("partition not used yet"); + let (_tx, rx, reservation) = state.channels.remove(&partition).expect("partition not used yet"); // Store streams from all the input partitions: let input_streams = rx @@ -601,8 +576,7 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { // input partitions to this partition: let fetch = None; let merge_reservation = - MemoryConsumer::new(format!("{}[Merge {partition}]", self.name())) - .register(context.memory_pool()); + MemoryConsumer::new(format!("{}[Merge {partition}]", self.name())).register(context.memory_pool()); streaming_merge( input_streams, self.schema(), @@ -612,9 +586,7 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { fetch, merge_reservation, ) - } - } /// This struct converts a receiver to a stream. @@ -637,16 +609,11 @@ struct PerPartitionStream { impl Stream for PerPartitionStream { type Item = Result; - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { match self.receiver.recv().poll_unpin(cx) { Poll::Ready(Some(Some(v))) => { if let Ok(batch) = &v { - self.reservation - .lock() - .shrink(batch.get_array_memory_size()); + self.reservation.lock().shrink(batch.get_array_memory_size()); } Poll::Ready(Some(v)) } diff --git a/rust/lakesoul-io/src/sorted_merge/merge_operator.rs b/rust/lakesoul-io/src/sorted_merge/merge_operator.rs index bb5bc6bf6..df25656ef 100644 --- a/rust/lakesoul-io/src/sorted_merge/merge_operator.rs +++ b/rust/lakesoul-io/src/sorted_merge/merge_operator.rs @@ -201,8 +201,7 @@ macro_rules! sum_with_primitive_type_and_append_value { let offset = arr.offset(); let null_buf_range = buffer.slice(offset + range.begin_row, range.end_row - range.begin_row); // the entire range is null - is_none &= - null_buf_range.null_count() == (range.end_row - range.begin_row); + is_none &= null_buf_range.null_count() == (range.end_row - range.begin_row); } None => is_none = false, } diff --git a/rust/lakesoul-io/src/sorted_merge/mod.rs b/rust/lakesoul-io/src/sorted_merge/mod.rs index 84179b0be..c183bf8b9 100644 --- a/rust/lakesoul-io/src/sorted_merge/mod.rs +++ b/rust/lakesoul-io/src/sorted_merge/mod.rs @@ -2,7 +2,7 @@ // // SPDX-License-Identifier: Apache-2.0 -pub mod sorted_stream_merger; pub mod combiner; +pub mod merge_operator; pub mod sort_key_range; -pub mod merge_operator; \ No newline at end of file +pub mod sorted_stream_merger; diff --git a/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs b/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs index 9c9cfb815..dbfa623bf 100644 --- a/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs +++ b/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs @@ -671,7 +671,6 @@ mod tests { Field::new("c", DataType::Int32, true), ]); - let merge_stream = SortedStreamMerger::new_from_streams( vec![s1, s2, s3], Arc::new(schema), diff --git a/rust/lakesoul-metadata-c/src/lib.rs b/rust/lakesoul-metadata-c/src/lib.rs index 2f0775f3e..6603a30eb 100644 --- a/rust/lakesoul-metadata-c/src/lib.rs +++ b/rust/lakesoul-metadata-c/src/lib.rs @@ -7,14 +7,14 @@ extern crate core; use core::ffi::c_ptrdiff_t; +use std::ffi::{c_char, c_uchar, CStr, CString}; use std::io::Write; use std::ptr::NonNull; -use std::ffi::{c_char, c_uchar, CString, CStr}; -use lakesoul_metadata::{Runtime, Builder, Client, PreparedStatementMap, MetaDataClient}; +use lakesoul_metadata::{Builder, Client, MetaDataClient, PreparedStatementMap, Runtime}; use prost::bytes::BufMut; -use proto::proto::entity; use prost::Message; +use proto::proto::entity; #[repr(C)] pub struct Result { @@ -54,7 +54,6 @@ pub struct PreparedStatement { private: [u8; 0], } - #[repr(C)] pub struct TokioPostgresClient { private: [u8; 0], @@ -70,7 +69,6 @@ pub struct BytesResult { private: [u8; 0], } - fn convert_to_opaque_raw(obj: F) -> *mut T { Box::into_raw(Box::new(obj)) as *mut T } @@ -88,9 +86,7 @@ fn from_nonnull(obj: NonNull) -> T { } fn string_from_ptr(ptr: *const c_char) -> String { - unsafe { - CStr::from_ptr(ptr).to_str().unwrap().to_string() - } + unsafe { CStr::from_ptr(ptr).to_str().unwrap().to_string() } } pub type ResultCallback = extern "C" fn(T, *const c_char); @@ -105,23 +101,17 @@ pub extern "C" fn execute_insert( addr: c_ptrdiff_t, len: i32, ) { - let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()}; - let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut()}; - let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()}; + let runtime = unsafe { NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref() }; + let client = unsafe { NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut() }; + let prepared = unsafe { NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut() }; - let raw_parts = unsafe {std::slice::from_raw_parts(addr as *const u8, len as usize)}; + let raw_parts = unsafe { std::slice::from_raw_parts(addr as *const u8, len as usize) }; let wrapper = entity::JniWrapper::decode(prost::bytes::Bytes::from(raw_parts)).unwrap(); - let result = runtime.block_on(async { - lakesoul_metadata::execute_insert( - client, - prepared, - insert_type, - wrapper - ).await - }); + let result = + runtime.block_on(async { lakesoul_metadata::execute_insert(client, prepared, insert_type, wrapper).await }); match result { Ok(count) => callback(count, CString::new("").unwrap().into_raw()), - Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()) + Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()), } } @@ -132,23 +122,18 @@ pub extern "C" fn execute_update( client: NonNull>, prepared: NonNull>, update_type: i32, - joined_string: *const c_char + joined_string: *const c_char, ) { - let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()}; - let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut()}; - let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()}; + let runtime = unsafe { NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref() }; + let client = unsafe { NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut() }; + let prepared = unsafe { NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut() }; let result = runtime.block_on(async { - lakesoul_metadata::execute_update( - client, - prepared, - update_type, - string_from_ptr(joined_string), - ).await + lakesoul_metadata::execute_update(client, prepared, update_type, string_from_ptr(joined_string)).await }); match result { Ok(count) => callback(count, CString::new("").unwrap().into_raw()), - Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()) + Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()), } } @@ -159,29 +144,31 @@ pub extern "C" fn execute_query_scalar( client: NonNull>, prepared: NonNull>, update_type: i32, - joined_string: *const c_char + joined_string: *const c_char, ) { - let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()}; - let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut()}; - let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()}; + let runtime = unsafe { NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref() }; + let client = unsafe { NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut() }; + let prepared = unsafe { NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut() }; let result = runtime.block_on(async { - lakesoul_metadata::execute_query_scalar( - client, - prepared, - update_type, - string_from_ptr(joined_string), - ).await + lakesoul_metadata::execute_query_scalar(client, prepared, update_type, string_from_ptr(joined_string)).await }); match result { - Ok(Some(result)) => callback(CString::new(result.as_str()).unwrap().into_raw(), CString::new("").unwrap().into_raw()), - Ok(None) => callback(CString::new("").unwrap().into_raw(), CString::new("").unwrap().into_raw()), - Err(e) => callback(CString::new("").unwrap().into_raw(), CString::new(e.to_string().as_str()).unwrap().into_raw()) + Ok(Some(result)) => callback( + CString::new(result.as_str()).unwrap().into_raw(), + CString::new("").unwrap().into_raw(), + ), + Ok(None) => callback( + CString::new("").unwrap().into_raw(), + CString::new("").unwrap().into_raw(), + ), + Err(e) => callback( + CString::new("").unwrap().into_raw(), + CString::new(e.to_string().as_str()).unwrap().into_raw(), + ), } } - - #[no_mangle] pub extern "C" fn execute_query( callback: extern "C" fn(i32, *const c_char), @@ -191,26 +178,21 @@ pub extern "C" fn execute_query( query_type: i32, joined_string: *const c_char, ) -> NonNull> { - let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()}; - let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_ref()}; - let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()}; - + let runtime = unsafe { NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref() }; + let client = unsafe { NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_ref() }; + let prepared = unsafe { NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut() }; + let result = runtime.block_on(async { - lakesoul_metadata::execute_query( - client, - prepared, - query_type, - string_from_ptr(joined_string), - ).await + lakesoul_metadata::execute_query(client, prepared, query_type, string_from_ptr(joined_string)).await }); match result { Ok(u8_vec) => { let len = u8_vec.len(); - callback( len as i32, CString::new("").unwrap().into_raw()); + callback(len as i32, CString::new("").unwrap().into_raw()); convert_to_nonnull(Result::::new::>(u8_vec)) } Err(e) => { - callback(-1 , CString::new(e.to_string().as_str()).unwrap().into_raw()); + callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()); convert_to_nonnull(Result::::new::>(vec![])) } } @@ -224,22 +206,25 @@ pub extern "C" fn export_bytes_result( addr: c_ptrdiff_t, ) { let len = len as usize; - let bytes = unsafe {NonNull::new_unchecked(bytes.as_ref().ptr as *mut Vec).as_mut()}; + let bytes = unsafe { NonNull::new_unchecked(bytes.as_ref().ptr as *mut Vec).as_mut() }; if bytes.len() != len { - callback( false, CString::new("Size of buffer and result mismatch at export_bytes_result.").unwrap().into_raw()); + callback( + false, + CString::new("Size of buffer and result mismatch at export_bytes_result.") + .unwrap() + .into_raw(), + ); return; } bytes.push(0u8); bytes.shrink_to_fit(); - - let dst = unsafe { - std::slice::from_raw_parts_mut(addr as *mut u8, len + 1) - }; + + let dst = unsafe { std::slice::from_raw_parts_mut(addr as *mut u8, len + 1) }; let mut writer = dst.writer(); let _ = writer.write_all(bytes.as_slice()); - - callback( true, CString::new("").unwrap().into_raw()); + + callback(true, CString::new("").unwrap().into_raw()); } #[no_mangle] @@ -247,28 +232,24 @@ pub extern "C" fn free_bytes_result(bytes: NonNull>) { from_nonnull(bytes).free::>(); } - - #[no_mangle] pub extern "C" fn clean_meta_for_test( callback: extern "C" fn(i32, *const c_char), runtime: NonNull>, client: NonNull>, ) { - let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()}; - let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_ref()}; - let result = runtime.block_on(async{ - lakesoul_metadata::clean_meta_for_test(client).await - }); + let runtime = unsafe { NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref() }; + let client = unsafe { NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_ref() }; + let result = runtime.block_on(async { lakesoul_metadata::clean_meta_for_test(client).await }); match result { Ok(count) => callback(count, CString::new("").unwrap().into_raw()), - Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()) - } + Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw()), + } } #[no_mangle] pub extern "C" fn create_tokio_runtime() -> NonNull> { - let runtime = Builder::new_multi_thread() + let runtime = Builder::new_multi_thread() .enable_all() .worker_threads(2) .max_blocking_threads(8) @@ -288,13 +269,11 @@ pub extern "C" fn create_tokio_postgres_client( config: *const c_char, runtime: NonNull>, ) -> NonNull> { - let config = string_from_ptr(config); - let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()}; + let config = string_from_ptr(config); + let runtime = unsafe { NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref() }; + + let result = runtime.block_on(async { lakesoul_metadata::create_connection(config).await }); - let result = runtime.block_on(async{ - lakesoul_metadata::create_connection(config).await - }); - let result = match result { Ok(client) => { callback(true, CString::new("").unwrap().into_raw()); @@ -333,4 +312,4 @@ pub extern "C" fn create_lakesoul_metadata_client() -> NonNull>) { from_nonnull(prepared).free::(); -} \ No newline at end of file +} diff --git a/rust/lakesoul-metadata/src/error.rs b/rust/lakesoul-metadata/src/error.rs index 74b0dfeae..8e38b6fd9 100644 --- a/rust/lakesoul-metadata/src/error.rs +++ b/rust/lakesoul-metadata/src/error.rs @@ -2,7 +2,12 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::{result, sync::Arc, error::Error, io, fmt::{Display, Formatter}, num}; +use std::{ + error::Error, + fmt::{Display, Formatter}, + io, num, result, + sync::Arc, +}; /// Result type for operations that could result in an [LakeSoulMetaDataError] pub type Result = result::Result; @@ -84,20 +89,22 @@ impl Display for LakeSoulMetaDataError { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match *self { LakeSoulMetaDataError::PostgresError(ref desc) => write!(f, "postgres error: {desc}"), - LakeSoulMetaDataError::IoError(ref desc) => write!(f, "IO error: {desc}"), - LakeSoulMetaDataError::SerdeJsonError(ref desc) => write!(f, "serde_json error: {desc}"), - LakeSoulMetaDataError::ParseIntError(ref desc) => write!(f, "parse_int error: {desc}"), - LakeSoulMetaDataError::UuidError(ref desc) => write!(f, "uuid error: {desc}"), - LakeSoulMetaDataError::ProstEncodeError(ref desc) => write!(f, "prost encode error: {desc}"), - LakeSoulMetaDataError::ProstDecodeError(ref desc) => write!(f, "prost decode error: {desc}"), + LakeSoulMetaDataError::IoError(ref desc) => write!(f, "IO error: {desc}"), + LakeSoulMetaDataError::SerdeJsonError(ref desc) => write!(f, "serde_json error: {desc}"), + LakeSoulMetaDataError::ParseIntError(ref desc) => write!(f, "parse_int error: {desc}"), + LakeSoulMetaDataError::UuidError(ref desc) => write!(f, "uuid error: {desc}"), + LakeSoulMetaDataError::ProstEncodeError(ref desc) => write!(f, "prost encode error: {desc}"), + LakeSoulMetaDataError::ProstDecodeError(ref desc) => write!(f, "prost decode error: {desc}"), LakeSoulMetaDataError::Other(ref desc) => { write!(f, "Other error: {desc}") } LakeSoulMetaDataError::Internal(ref desc) => { - write!(f, "Internal error: {desc}.\nThis was likely caused by a bug in LakeSoul's \ - code and we would welcome that you file an bug report in our issue tracker") + write!( + f, + "Internal error: {desc}.\nThis was likely caused by a bug in LakeSoul's \ + code and we would welcome that you file an bug report in our issue tracker" + ) } - } } } diff --git a/rust/lakesoul-metadata/src/lib.rs b/rust/lakesoul-metadata/src/lib.rs index e6b7dbe26..81af2d5b5 100644 --- a/rust/lakesoul-metadata/src/lib.rs +++ b/rust/lakesoul-metadata/src/lib.rs @@ -2,36 +2,34 @@ // // SPDX-License-Identifier: Apache-2.0 -mod metadata_client; pub mod error; +mod metadata_client; -use std::{collections::HashMap, io::ErrorKind}; use std::str::FromStr; +use std::{collections::HashMap, io::ErrorKind}; use error::{LakeSoulMetaDataError, Result}; -use proto::proto::entity; use prost::Message; +use proto::proto::entity; pub use tokio::runtime::{Builder, Runtime}; +use postgres_types::{FromSql, ToSql}; use tokio::spawn; -pub use tokio_postgres::{NoTls, Client, Statement}; -use postgres_types::{ToSql, FromSql}; +pub use tokio_postgres::{Client, NoTls, Statement}; pub use metadata_client::{MetaDataClient, MetaDataClientRef}; -pub const DAO_TYPE_QUERY_ONE_OFFSET : i32 = 0; -pub const DAO_TYPE_QUERY_LIST_OFFSET : i32 = 100; -pub const DAO_TYPE_INSERT_ONE_OFFSET : i32 = 200; -pub const DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET : i32 = 300; -pub const DAO_TYPE_QUERY_SCALAR_OFFSET : i32 = 400; -pub const DAO_TYPE_UPDATE_OFFSET : i32 = 500; +pub const DAO_TYPE_QUERY_ONE_OFFSET: i32 = 0; +pub const DAO_TYPE_QUERY_LIST_OFFSET: i32 = 100; +pub const DAO_TYPE_INSERT_ONE_OFFSET: i32 = 200; +pub const DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET: i32 = 300; +pub const DAO_TYPE_QUERY_SCALAR_OFFSET: i32 = 400; +pub const DAO_TYPE_UPDATE_OFFSET: i32 = 500; pub const PARAM_DELIM: &str = "__DELIM__"; pub const PARTITION_DESC_DELIM: &str = "_DELIM_"; - - enum ResultType { Namespace, TableInfo, @@ -54,14 +52,15 @@ struct DataFileOp { } impl DataFileOp { - fn from_proto_data_file_op( - data_file_op: &entity::DataFileOp - ) -> Self { - DataFileOp{ + fn from_proto_data_file_op(data_file_op: &entity::DataFileOp) -> Self { + DataFileOp { path: data_file_op.path.clone(), - file_op: proto::proto::entity::FileOp::from_i32(data_file_op.file_op).unwrap().as_str_name().to_string(), + file_op: proto::proto::entity::FileOp::from_i32(data_file_op.file_op) + .unwrap() + .as_str_name() + .to_string(), size: data_file_op.size, - file_exist_cols: data_file_op.file_exist_cols.clone() + file_exist_cols: data_file_op.file_exist_cols.clone(), } } @@ -70,14 +69,14 @@ impl DataFileOp { path: self.path.clone(), file_op: proto::proto::entity::FileOp::from_str_name(self.file_op.as_str()).unwrap() as i32, size: self.size, - file_exist_cols: self.file_exist_cols.clone() + file_exist_cols: self.file_exist_cols.clone(), } } } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, num_enum::TryFromPrimitive)] #[repr(i32)] -pub enum DaoType{ +pub enum DaoType { SelectNamespaceByNamespace = DAO_TYPE_QUERY_ONE_OFFSET, SelectTablePathIdByTablePath = DAO_TYPE_QUERY_ONE_OFFSET + 1, SelectTableInfoByTableId = DAO_TYPE_QUERY_ONE_OFFSET + 2, @@ -92,7 +91,6 @@ pub enum DaoType{ SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId = DAO_TYPE_QUERY_ONE_OFFSET + 9, // ==== Query List ==== - ListNamespaces = DAO_TYPE_QUERY_LIST_OFFSET, ListTableNameByNamespace = DAO_TYPE_QUERY_LIST_OFFSET + 1, ListAllTablePath = DAO_TYPE_QUERY_LIST_OFFSET + 2, @@ -154,13 +152,11 @@ pub enum DaoType{ DeleteDataCommitInfoByTableId = DAO_TYPE_UPDATE_OFFSET + 15, } - pub type PreparedStatementMap = HashMap; - async fn get_prepared_statement( client: &Client, - prepared :&mut PreparedStatementMap, + prepared: &mut PreparedStatementMap, dao_type: &DaoType, ) -> Result { if let Some(statement) = prepared.get(dao_type) { @@ -406,7 +402,7 @@ async fn get_prepared_statement( prepared.insert(*dao_type, statement.clone()); Ok(statement) } - Err(err) => Err(LakeSoulMetaDataError::from(err)) + Err(err) => Err(LakeSoulMetaDataError::from(err)), } } } @@ -414,8 +410,8 @@ async fn get_prepared_statement( pub async fn execute_query( client: &Client, prepared: &mut PreparedStatementMap, - query_type: i32, - joined_string: String, + query_type: i32, + joined_string: String, ) -> Result> { if query_type >= DAO_TYPE_INSERT_ONE_OFFSET { eprintln!("Invalid query_type_index: {:?}", query_type); @@ -428,12 +424,11 @@ pub async fn execute_query( .split(PARAM_DELIM) .collect::>() .iter() - .map(|str|str.to_string()) + .map(|str| str.to_string()) .collect::>(); let rows = match query_type { - DaoType::ListNamespaces | - DaoType::ListAllTablePath if params.len() == 1 && params[0].is_empty() => { + DaoType::ListNamespaces | DaoType::ListAllTablePath if params.len() == 1 && params[0].is_empty() => { let result = client.query(&statement, &[]).await; match result { Ok(rows) => rows, @@ -447,10 +442,12 @@ pub async fn execute_query( Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } - DaoType::SelectNamespaceByNamespace | - DaoType::SelectTableInfoByTableId | - DaoType::SelectTablePathIdByTablePath | - DaoType::SelectTableInfoByTablePath if params.len() == 1 => { + DaoType::SelectNamespaceByNamespace + | DaoType::SelectTableInfoByTableId + | DaoType::SelectTablePathIdByTablePath + | DaoType::SelectTableInfoByTablePath + if params.len() == 1 => + { let result = client.query_opt(&statement, &[¶ms[0]]).await; match result { Ok(Some(row)) => vec![row], @@ -458,25 +455,27 @@ pub async fn execute_query( Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } - DaoType::ListPartitionByTableId | - DaoType::ListAllPathTablePathByNamespace if params.len() == 1 => { + DaoType::ListPartitionByTableId | DaoType::ListAllPathTablePathByNamespace if params.len() == 1 => { let result = client.query(&statement, &[¶ms[0]]).await; match result { Ok(rows) => rows, Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } - DaoType::SelectOnePartitionVersionByTableIdAndDesc | - DaoType::ListPartitionByTableIdAndDesc if params.len() == 2 => { + DaoType::SelectOnePartitionVersionByTableIdAndDesc | DaoType::ListPartitionByTableIdAndDesc + if params.len() == 2 => + { let result = client.query(&statement, &[¶ms[0], ¶ms[1]]).await; match result { Ok(rows) => rows, Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } - DaoType::SelectTableNameIdByTableName | - DaoType::SelectTableInfoByTableNameAndNameSpace | - DaoType::SelectTableInfoByIdAndTablePath if params.len() == 2 => { + DaoType::SelectTableNameIdByTableName + | DaoType::SelectTableInfoByTableNameAndNameSpace + | DaoType::SelectTableInfoByIdAndTablePath + if params.len() == 2 => + { let result = client.query_opt(&statement, &[¶ms[0], ¶ms[1]]).await; match result { Ok(Some(row)) => vec![row], @@ -485,7 +484,12 @@ pub async fn execute_query( } } DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId if params.len() == 3 => { - let result = client.query_opt(&statement, &[¶ms[0], ¶ms[1], &uuid::Uuid::from_str(¶ms[2])?]).await; + let result = client + .query_opt( + &statement, + &[¶ms[0], ¶ms[1], &uuid::Uuid::from_str(¶ms[2])?], + ) + .await; match result { Ok(Some(row)) => vec![row], Ok(None) => vec![], @@ -493,26 +497,42 @@ pub async fn execute_query( } } DaoType::SelectPartitionVersionByTableIdAndDescAndVersion if params.len() == 3 => { - let result = client.query(&statement, &[¶ms[0], ¶ms[1], &i32::from_str(¶ms[2])?]).await; + let result = client + .query(&statement, &[¶ms[0], ¶ms[1], &i32::from_str(¶ms[2])?]) + .await; match result { Ok(rows) => rows, Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } - DaoType::ListCommitOpsBetweenVersions | - DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange if params.len() == 4 => { - let result = client.query(&statement, &[¶ms[0], ¶ms[1], &i32::from_str(¶ms[2])?, &i32::from_str(¶ms[3])?]).await; + DaoType::ListCommitOpsBetweenVersions + | DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange + if params.len() == 4 => + { + let result = client + .query( + &statement, + &[ + ¶ms[0], + ¶ms[1], + &i32::from_str(¶ms[2])?, + &i32::from_str(¶ms[3])?, + ], + ) + .await; match result { Ok(rows) => rows, Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } DaoType::ListPartitionDescByTableIdAndParList if params.len() == 2 => { - let partitions = "'".to_owned() + ¶ms[1] - .replace('\'', "''") - .split(PARTITION_DESC_DELIM) - .collect::>() - .join("','") + "'"; + let partitions = "'".to_owned() + + ¶ms[1] + .replace('\'', "''") + .split(PARTITION_DESC_DELIM) + .collect::>() + .join("','") + + "'"; let statement = format!("select m.table_id, t.partition_desc, m.version, m.commit_op, m.snapshot, m.expression, m.domain from ( select table_id,partition_desc,max(version) from partition_info where table_id = $1::TEXT and partition_desc in ({}) @@ -528,7 +548,17 @@ pub async fn execute_query( } } DaoType::ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange if params.len() == 4 => { - let result = client.query(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2])?, &i64::from_str(¶ms[3])?]).await; + let result = client + .query( + &statement, + &[ + ¶ms[0], + ¶ms[1], + &i64::from_str(¶ms[2])?, + &i64::from_str(¶ms[3])?, + ], + ) + .await; match result { Ok(rows) => rows, Err(e) => return Err(LakeSoulMetaDataError::from(e)), @@ -538,14 +568,16 @@ pub async fn execute_query( let concated_uuid = ¶ms[2]; if concated_uuid.len() % 32 != 0 { eprintln!("Invalid params of query_type={:?}, params={:?}", query_type, params); - return Err(LakeSoulMetaDataError::from(std::io::Error::from(std::io::ErrorKind::InvalidInput))); + return Err(LakeSoulMetaDataError::from(std::io::Error::from( + std::io::ErrorKind::InvalidInput, + ))); } let uuid_num = concated_uuid.len() / 32; let mut uuid_list = Vec::::with_capacity(uuid_num); let mut idx = 0; for _ in 0..uuid_num { - let high = u64::from_str_radix(&concated_uuid[idx..idx+16], 16)?; - let low = u64::from_str_radix(&concated_uuid[idx+16..idx+32], 16)?; + let high = u64::from_str_radix(&concated_uuid[idx..idx + 16], 16)?; + let low = u64::from_str_radix(&concated_uuid[idx + 16..idx + 32], 16)?; uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string()); idx += 32; } @@ -554,12 +586,14 @@ pub async fn execute_query( let uuid_list_str = uuid_list.join(""); - - let statement = format!("select table_id, partition_desc, commit_id, file_ops, commit_op, timestamp, committed, domain + let statement = format!( + "select table_id, partition_desc, commit_id, file_ops, commit_op, timestamp, committed, domain from data_commit_info where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id in ({}) - order by position(commit_id::text in '{}')", uuid_str_list, uuid_list_str); + order by position(commit_id::text in '{}')", + uuid_str_list, uuid_list_str + ); let result = { let statement = client.prepare(&statement).await?; @@ -575,35 +609,32 @@ pub async fn execute_query( return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } }; - + let result_type = match query_type { - DaoType::SelectNamespaceByNamespace | - DaoType::ListNamespaces => ResultType::Namespace, + DaoType::SelectNamespaceByNamespace | DaoType::ListNamespaces => ResultType::Namespace, - DaoType::SelectTableInfoByTableId | - DaoType::SelectTableInfoByTableNameAndNameSpace | - DaoType::SelectTableInfoByTablePath | - DaoType::SelectTableInfoByIdAndTablePath => ResultType::TableInfo, + DaoType::SelectTableInfoByTableId + | DaoType::SelectTableInfoByTableNameAndNameSpace + | DaoType::SelectTableInfoByTablePath + | DaoType::SelectTableInfoByIdAndTablePath => ResultType::TableInfo, - DaoType::SelectTablePathIdByTablePath | - DaoType::ListAllTablePath => ResultType::TablePathId, + DaoType::SelectTablePathIdByTablePath | DaoType::ListAllTablePath => ResultType::TablePathId, - DaoType::SelectTableNameIdByTableName | - DaoType::ListTableNameByNamespace => ResultType::TableNameId, + DaoType::SelectTableNameIdByTableName | DaoType::ListTableNameByNamespace => ResultType::TableNameId, - DaoType::ListPartitionByTableId | - DaoType::ListPartitionDescByTableIdAndParList | - DaoType::SelectPartitionVersionByTableIdAndDescAndVersion | - DaoType::SelectOnePartitionVersionByTableIdAndDesc => ResultType::PartitionInfoWithoutTimestamp, + DaoType::ListPartitionByTableId + | DaoType::ListPartitionDescByTableIdAndParList + | DaoType::SelectPartitionVersionByTableIdAndDescAndVersion + | DaoType::SelectOnePartitionVersionByTableIdAndDesc => ResultType::PartitionInfoWithoutTimestamp, - DaoType::ListPartitionByTableIdAndDesc | - DaoType::ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange | - DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange => ResultType::PartitionInfo, + DaoType::ListPartitionByTableIdAndDesc + | DaoType::ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange + | DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange => ResultType::PartitionInfo, - DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId | - DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList => ResultType::DataCommitInfo, + DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId + | DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList => ResultType::DataCommitInfo, - DaoType::ListAllPathTablePathByNamespace => ResultType::TablePathIdWithOnlyPath , + DaoType::ListAllPathTablePathByNamespace => ResultType::TablePathIdWithOnlyPath, DaoType::ListCommitOpsBetweenVersions => ResultType::PartitionInfoWithOnlyCommitOp, _ => { @@ -614,309 +645,305 @@ pub async fn execute_query( let wrapper = match result_type { ResultType::TableNameId => { - let table_name_id :Vec = - rows - .iter() - .map(|row|proto::proto::entity::TableNameId { - table_name: row.get(0), - table_id: row.get(1), - table_namespace: row.get(2), - domain: row.get(3), - }) - .collect(); + let table_name_id: Vec = rows + .iter() + .map(|row| proto::proto::entity::TableNameId { + table_name: row.get(0), + table_id: row.get(1), + table_namespace: row.get(2), + domain: row.get(3), + }) + .collect(); proto::proto::entity::JniWrapper { - table_name_id, - ..Default::default() + table_name_id, + ..Default::default() } } ResultType::TablePathId => { - let table_path_id :Vec = - rows - .iter() - .map(|row|proto::proto::entity::TablePathId { - table_path: row.get(0), - table_id: row.get(1), - table_namespace: row.get(2), - domain: row.get(3), - }) - .collect(); + let table_path_id: Vec = rows + .iter() + .map(|row| proto::proto::entity::TablePathId { + table_path: row.get(0), + table_id: row.get(1), + table_namespace: row.get(2), + domain: row.get(3), + }) + .collect(); proto::proto::entity::JniWrapper { - table_path_id, - ..Default::default() + table_path_id, + ..Default::default() } } ResultType::TablePathIdWithOnlyPath => { - let table_path_id :Vec = - rows - .iter() - .map(|row|proto::proto::entity::TablePathId { - table_path: row.get(0), - ..Default::default() - }) - .collect(); + let table_path_id: Vec = rows + .iter() + .map(|row| proto::proto::entity::TablePathId { + table_path: row.get(0), + ..Default::default() + }) + .collect(); proto::proto::entity::JniWrapper { - table_path_id, - ..Default::default() + table_path_id, + ..Default::default() } } ResultType::Namespace => { - let namespace:Vec = - rows - .iter() - .map(|row|proto::proto::entity::Namespace { - namespace: row.get(0), - properties: row.get::<_, serde_json::Value>(1).to_string(), - comment: row.get::<_, Option>(2).unwrap_or(String::from("")), - domain: row.get(3) - }) - .collect(); + let namespace: Vec = rows + .iter() + .map(|row| proto::proto::entity::Namespace { + namespace: row.get(0), + properties: row.get::<_, serde_json::Value>(1).to_string(), + comment: row.get::<_, Option>(2).unwrap_or(String::from("")), + domain: row.get(3), + }) + .collect(); proto::proto::entity::JniWrapper { namespace, - ..Default::default() + ..Default::default() } - }, + } ResultType::TableInfo => { - let table_info:Vec = - rows - .iter() - .map(|row|proto::proto::entity::TableInfo { - table_id: row.get(0), - table_name: row.get(1), - table_path: row.get(2), - table_schema: row.get(3), - properties: row.get::<_, serde_json::Value>(4).to_string(), - partitions: row.get(5), - table_namespace: row.get(6), - domain: row.get(7) - }) - .collect(); + let table_info: Vec = rows + .iter() + .map(|row| proto::proto::entity::TableInfo { + table_id: row.get(0), + table_name: row.get(1), + table_path: row.get(2), + table_schema: row.get(3), + properties: row.get::<_, serde_json::Value>(4).to_string(), + partitions: row.get(5), + table_namespace: row.get(6), + domain: row.get(7), + }) + .collect(); proto::proto::entity::JniWrapper { table_info, - ..Default::default() + ..Default::default() } } ResultType::PartitionInfo => { - let partition_info:Vec = - rows - .iter() - .map(|row|{ - proto::proto::entity::PartitionInfo { - table_id: row.get(0), - partition_desc: row.get(1), - version: row.get::<_, i32>(2), - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, - snapshot: row.get::<_, Vec>(4) - .iter() - .map(|uuid| { - let (high, low) = uuid.as_u64_pair(); - entity::Uuid{high, low} - }) - .collect::>(), - timestamp: row.get::<_, i64>(5), - expression: row.get::<_, Option>(6).unwrap_or(String::from("")), - domain: row.get(7), - } - }) - .collect(); + let partition_info: Vec = rows + .iter() + .map(|row| proto::proto::entity::PartitionInfo { + table_id: row.get(0), + partition_desc: row.get(1), + version: row.get::<_, i32>(2), + commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, + snapshot: row + .get::<_, Vec>(4) + .iter() + .map(|uuid| { + let (high, low) = uuid.as_u64_pair(); + entity::Uuid { high, low } + }) + .collect::>(), + timestamp: row.get::<_, i64>(5), + expression: row.get::<_, Option>(6).unwrap_or(String::from("")), + domain: row.get(7), + }) + .collect(); proto::proto::entity::JniWrapper { partition_info, - ..Default::default() + ..Default::default() } } ResultType::PartitionInfoWithoutTimestamp => { - let partition_info:Vec = - rows - .iter() - .map(|row|{ - proto::proto::entity::PartitionInfo { - table_id: row.get(0), - partition_desc: row.get(1), - version: row.get::<_, i32>(2), - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, - snapshot: row.get::<_, Vec>(4) - .iter() - .map(|uuid| { - let (high, low) = uuid.as_u64_pair(); - entity::Uuid{high, low} - }) - .collect::>(), - expression: row.get::<_, Option>(5).unwrap_or(String::from("")), - domain: row.get(6), - ..Default::default() - } - }) - .collect(); + let partition_info: Vec = rows + .iter() + .map(|row| proto::proto::entity::PartitionInfo { + table_id: row.get(0), + partition_desc: row.get(1), + version: row.get::<_, i32>(2), + commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, + snapshot: row + .get::<_, Vec>(4) + .iter() + .map(|uuid| { + let (high, low) = uuid.as_u64_pair(); + entity::Uuid { high, low } + }) + .collect::>(), + expression: row.get::<_, Option>(5).unwrap_or(String::from("")), + domain: row.get(6), + ..Default::default() + }) + .collect(); proto::proto::entity::JniWrapper { partition_info, - ..Default::default() + ..Default::default() } } ResultType::PartitionInfoWithOnlyCommitOp => { - let partition_info:Vec = - rows - .iter() - .map(|row|{ - proto::proto::entity::PartitionInfo { - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(0)).unwrap() as i32, - ..Default::default() - } - }) - .collect(); + let partition_info: Vec = rows + .iter() + .map(|row| proto::proto::entity::PartitionInfo { + commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(0)).unwrap() as i32, + ..Default::default() + }) + .collect(); proto::proto::entity::JniWrapper { partition_info, - ..Default::default() + ..Default::default() } } ResultType::DataCommitInfo => { - let data_commit_info:Vec = - rows - .iter() - .map(|row|{ - proto::proto::entity::DataCommitInfo { - table_id: row.get(0), - partition_desc: row.get(1), - commit_id: { - let (high, low)=row.get::<_, uuid::Uuid>(2).as_u64_pair(); - Some(entity::Uuid{high, low}) - }, - file_ops: row.get::<_, Vec>(3) - .iter() - .map(|data_file_op| data_file_op.as_proto_data_file_op()) - .collect::>(), - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(4)).unwrap() as i32, - timestamp: row.get(5), - committed: row.get(6), - domain: row.get(7), - } - }) - .collect(); + let data_commit_info: Vec = rows + .iter() + .map(|row| proto::proto::entity::DataCommitInfo { + table_id: row.get(0), + partition_desc: row.get(1), + commit_id: { + let (high, low) = row.get::<_, uuid::Uuid>(2).as_u64_pair(); + Some(entity::Uuid { high, low }) + }, + file_ops: row + .get::<_, Vec>(3) + .iter() + .map(|data_file_op| data_file_op.as_proto_data_file_op()) + .collect::>(), + commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(4)).unwrap() as i32, + timestamp: row.get(5), + committed: row.get(6), + domain: row.get(7), + }) + .collect(); proto::proto::entity::JniWrapper { data_commit_info, - ..Default::default() + ..Default::default() } } }; Ok(wrapper.encode_to_vec()) } - pub async fn execute_insert( client: &mut Client, prepared: &mut PreparedStatementMap, - insert_type: i32, + insert_type: i32, wrapper: entity::JniWrapper, ) -> Result { - if !(DAO_TYPE_INSERT_ONE_OFFSET..DAO_TYPE_QUERY_SCALAR_OFFSET).contains(&insert_type){ + if !(DAO_TYPE_INSERT_ONE_OFFSET..DAO_TYPE_QUERY_SCALAR_OFFSET).contains(&insert_type) { eprintln!("Invalid insert_type_index: {:?}", insert_type); - return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)) + return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } let insert_type = DaoType::try_from(insert_type).unwrap(); let statement = get_prepared_statement(client, prepared, &insert_type).await?; let result = match insert_type { - DaoType::InsertNamespace if wrapper.namespace.len() == 1 => { - let namespace = wrapper.namespace.get(0).unwrap(); - let properties:serde_json::Value = serde_json::from_str(&namespace.properties)?; - client.execute( - &statement, - &[ - &namespace.namespace, - &properties, - &namespace.comment, - &namespace.domain, - ] - ).await + DaoType::InsertNamespace if wrapper.namespace.len() == 1 => { + let namespace = wrapper.namespace.first().unwrap(); + let properties: serde_json::Value = serde_json::from_str(&namespace.properties)?; + client + .execute( + &statement, + &[&namespace.namespace, &properties, &namespace.comment, &namespace.domain], + ) + .await } - DaoType::InsertTableInfo if wrapper.table_info.len() == 1=> { - let table_info = wrapper.table_info.get(0).unwrap(); - let properties:serde_json::Value = serde_json::from_str(&table_info.properties)?; - client.execute( - &statement, - &[ - &table_info.table_id, - &table_info.table_name, - &table_info.table_path, - &table_info.table_schema, - &properties, - &table_info.partitions, - &table_info.table_namespace, - &table_info.domain, - ] - ).await + DaoType::InsertTableInfo if wrapper.table_info.len() == 1 => { + let table_info = wrapper.table_info.first().unwrap(); + let properties: serde_json::Value = serde_json::from_str(&table_info.properties)?; + client + .execute( + &statement, + &[ + &table_info.table_id, + &table_info.table_name, + &table_info.table_path, + &table_info.table_schema, + &properties, + &table_info.partitions, + &table_info.table_namespace, + &table_info.domain, + ], + ) + .await } DaoType::InsertTableNameId if wrapper.table_name_id.len() == 1 => { - let table_name_id = wrapper.table_name_id.get(0).unwrap(); - client.execute( - &statement, - &[ - &table_name_id.table_id, - &table_name_id.table_name, - &table_name_id.table_namespace, - &table_name_id.domain, - ] - ).await + let table_name_id = wrapper.table_name_id.first().unwrap(); + client + .execute( + &statement, + &[ + &table_name_id.table_id, + &table_name_id.table_name, + &table_name_id.table_namespace, + &table_name_id.domain, + ], + ) + .await } DaoType::InsertTablePathId if wrapper.table_path_id.len() == 1 => { - let table_path_id = wrapper.table_path_id.get(0).unwrap(); - client.execute( - &statement, - &[ - &table_path_id.table_id, - &table_path_id.table_path, - &table_path_id.table_namespace, - &table_path_id.domain, - ] - ).await + let table_path_id = wrapper.table_path_id.first().unwrap(); + client + .execute( + &statement, + &[ + &table_path_id.table_id, + &table_path_id.table_path, + &table_path_id.table_namespace, + &table_path_id.domain, + ], + ) + .await } - DaoType::InsertPartitionInfo if wrapper.partition_info.len() == 1 =>{ - let partition_info = wrapper.partition_info.get(0).unwrap(); - let snapshot = partition_info.snapshot + DaoType::InsertPartitionInfo if wrapper.partition_info.len() == 1 => { + let partition_info = wrapper.partition_info.first().unwrap(); + let snapshot = partition_info + .snapshot .iter() .map(|_uuid| uuid::Uuid::from_u64_pair(_uuid.high, _uuid.low)) .collect::>(); - client.execute( - &statement, - &[ - &partition_info.table_id, - &partition_info.partition_desc, - &partition_info.version, - &partition_info.commit_op().as_str_name(), - &snapshot, - &partition_info.expression, - &partition_info.domain - ] - ).await + client + .execute( + &statement, + &[ + &partition_info.table_id, + &partition_info.partition_desc, + &partition_info.version, + &partition_info.commit_op().as_str_name(), + &snapshot, + &partition_info.expression, + &partition_info.domain, + ], + ) + .await } - DaoType::InsertDataCommitInfo if wrapper.data_commit_info.len() == 1 =>{ - let data_commit_info = wrapper.data_commit_info.get(0).unwrap(); - let file_ops = data_commit_info.file_ops + DaoType::InsertDataCommitInfo if wrapper.data_commit_info.len() == 1 => { + let data_commit_info = wrapper.data_commit_info.first().unwrap(); + let file_ops = data_commit_info + .file_ops .iter() .map(DataFileOp::from_proto_data_file_op) .collect::>(); let commit_id = data_commit_info.commit_id.as_ref().unwrap(); let _uuid = uuid::Uuid::from_u64_pair(commit_id.high, commit_id.low); - - client.execute( - &statement, - &[ - &data_commit_info.table_id, - &data_commit_info.partition_desc, - &_uuid, - &file_ops, - &data_commit_info.commit_op().as_str_name(), - &data_commit_info.timestamp, - &data_commit_info.committed, - &data_commit_info.domain - ] - ).await + + client + .execute( + &statement, + &[ + &data_commit_info.table_id, + &data_commit_info.partition_desc, + &_uuid, + &file_ops, + &data_commit_info.commit_op().as_str_name(), + &data_commit_info.timestamp, + &data_commit_info.committed, + &data_commit_info.domain, + ], + ) + .await } - DaoType::TransactionInsertPartitionInfo => { + DaoType::TransactionInsertPartitionInfo => { let partition_info_list = wrapper.partition_info; let result = { let transaction = client.transaction().await?; - let prepared = transaction.prepare("insert into partition_info( + let prepared = transaction + .prepare( + "insert into partition_info( table_id, partition_desc, version, @@ -925,72 +952,79 @@ pub async fn execute_insert( expression, domain ) - values($1::TEXT, $2::TEXT, $3::INT, $4::TEXT, $5::_UUID, $6::TEXT, $7::TEXT)").await; + values($1::TEXT, $2::TEXT, $3::INT, $4::TEXT, $5::_UUID, $6::TEXT, $7::TEXT)", + ) + .await; let statement = match prepared { Ok(statement) => statement, - Err(e) => return Err(LakeSoulMetaDataError::from(e)) + Err(e) => return Err(LakeSoulMetaDataError::from(e)), }; for i in 0..partition_info_list.len() { let partition_info = partition_info_list.get(i).unwrap(); - let snapshot = partition_info.snapshot + let snapshot = partition_info + .snapshot .iter() .map(|_uuid| uuid::Uuid::from_u64_pair(_uuid.high, _uuid.low)) .collect::>(); - - let result = transaction.execute( - &statement, - &[ - &partition_info.table_id, - &partition_info.partition_desc, - &partition_info.version, - &partition_info.commit_op().as_str_name(), - &snapshot, - &partition_info.expression, - &partition_info.domain - ] - ).await; - + + let result = transaction + .execute( + &statement, + &[ + &partition_info.table_id, + &partition_info.partition_desc, + &partition_info.version, + &partition_info.commit_op().as_str_name(), + &snapshot, + &partition_info.expression, + &partition_info.domain, + ], + ) + .await; + if let Some(e) = result.err() { eprintln!("transaction insert error, err = {:?}", e); - return match transaction.rollback().await{ + return match transaction.rollback().await { Ok(()) => Ok(0i32), - Err(e) => Err(LakeSoulMetaDataError::from(e)) + Err(e) => Err(LakeSoulMetaDataError::from(e)), }; }; for uuid in &snapshot { - let result = transaction.execute( - "update data_commit_info set committed = 'true' where commit_id = $1::UUID", - &[&uuid] - ).await; - + let result = transaction + .execute( + "update data_commit_info set committed = 'true' where commit_id = $1::UUID", + &[&uuid], + ) + .await; + if let Some(e) = result.err() { eprintln!("update committed error, err = {:?}", e); - return match transaction.rollback().await{ + return match transaction.rollback().await { Ok(()) => Ok(0i32), - Err(e) => Err(LakeSoulMetaDataError::from(e)) + Err(e) => Err(LakeSoulMetaDataError::from(e)), }; } - }; - }; - match transaction.commit().await{ + } + } + match transaction.commit().await { Ok(()) => Ok(partition_info_list.len() as u64), - Err(e) => Err(e) + Err(e) => Err(e), } }; match result { Ok(count) => Ok(count), - Err(e) => { - return Err(LakeSoulMetaDataError::from(e)) - } + Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } - DaoType::TransactionInsertDataCommitInfo => { + DaoType::TransactionInsertDataCommitInfo => { let data_commit_info_list = wrapper.data_commit_info; let result = { let transaction = client.transaction().await?; - let prepared = transaction.prepare("insert into data_commit_info( + let prepared = transaction + .prepare( + "insert into data_commit_info( table_id, partition_desc, commit_id, @@ -1000,59 +1034,61 @@ pub async fn execute_insert( committed, domain ) - values($1::TEXT, $2::TEXT, $3::UUID, $4::_data_file_op, $5::TEXT, $6::BIGINT, $7::BOOL, $8::TEXT)").await; + values($1::TEXT, $2::TEXT, $3::UUID, $4::_data_file_op, $5::TEXT, $6::BIGINT, $7::BOOL, $8::TEXT)", + ) + .await; let statement = match prepared { Ok(statement) => statement, - Err(e) => return Err(LakeSoulMetaDataError::from(e)) + Err(e) => return Err(LakeSoulMetaDataError::from(e)), }; for i in 0..data_commit_info_list.len() { let data_commit_info = data_commit_info_list.get(i).unwrap(); - let file_ops = data_commit_info.file_ops + let file_ops = data_commit_info + .file_ops .iter() .map(DataFileOp::from_proto_data_file_op) .collect::>(); let commit_id = data_commit_info.commit_id.as_ref().unwrap(); let _uuid = uuid::Uuid::from_u64_pair(commit_id.high, commit_id.low); - - let result = transaction.execute( - &statement, - &[ - &data_commit_info.table_id, - &data_commit_info.partition_desc, - &_uuid, - &file_ops, - &data_commit_info.commit_op().as_str_name(), - &data_commit_info.timestamp, - &data_commit_info.committed, - &data_commit_info.domain - ] - ).await; - + + let result = transaction + .execute( + &statement, + &[ + &data_commit_info.table_id, + &data_commit_info.partition_desc, + &_uuid, + &file_ops, + &data_commit_info.commit_op().as_str_name(), + &data_commit_info.timestamp, + &data_commit_info.committed, + &data_commit_info.domain, + ], + ) + .await; + if let Some(e) = result.err() { eprintln!("transaction insert error, err = {:?}", e); - return match transaction.rollback().await{ + return match transaction.rollback().await { Ok(()) => Ok(0i32), - Err(e) => Err(LakeSoulMetaDataError::from(e)) + Err(e) => Err(LakeSoulMetaDataError::from(e)), }; }; - - }; - match transaction.commit().await{ + } + match transaction.commit().await { Ok(()) => Ok(data_commit_info_list.len() as u64), - Err(e) => Err(e) + Err(e) => Err(e), } }; match result { Ok(count) => Ok(count), - Err(e) => { - return Err(LakeSoulMetaDataError::from(e)) - } + Err(e) => return Err(LakeSoulMetaDataError::from(e)), } } _ => { eprintln!("InvalidInput of type={:?}: {:?}", insert_type, wrapper); - return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)) + return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } }; match result { @@ -1064,39 +1100,44 @@ pub async fn execute_insert( pub async fn execute_update( client: &mut Client, prepared: &mut PreparedStatementMap, - update_type: i32, - joined_string: String, + update_type: i32, + joined_string: String, ) -> Result { if update_type < DAO_TYPE_UPDATE_OFFSET { eprintln!("Invalid update_type_index: {:?}", update_type); - return Err(LakeSoulMetaDataError::from(std::io::ErrorKind::InvalidInput)) + return Err(LakeSoulMetaDataError::from(std::io::ErrorKind::InvalidInput)); } let update_type = DaoType::try_from(update_type).unwrap(); - let statement = get_prepared_statement( client, prepared, &update_type).await?; + let statement = get_prepared_statement(client, prepared, &update_type).await?; let params = joined_string .split(PARAM_DELIM) .collect::>() .iter() - .map(|str|str.to_string()) + .map(|str| str.to_string()) .collect::>(); let result = match update_type { - DaoType::DeleteNamespaceByNamespace | - DaoType::DeletePartitionInfoByTableId | - DaoType::DeleteDataCommitInfoByTableId | - DaoType::DeleteTableNameIdByTableId | - DaoType::DeleteTablePathIdByTableId | - DaoType::DeleteTablePathIdByTablePath if params.len() == 1 => - client.execute(&statement, &[¶ms[0]]).await, - DaoType::DeleteTableInfoByIdAndPath | - DaoType::DeleteTableNameIdByTableNameAndNamespace | - DaoType::DeletePartitionInfoByTableIdAndPartitionDesc | - DaoType::DeleteDataCommitInfoByTableIdAndPartitionDesc if params.len() == 2 => - client.execute(&statement, &[¶ms[0], ¶ms[1]]).await, - DaoType::UpdateTableInfoPropertiesById | - DaoType::UpdateNamespacePropertiesByNamespace if params.len() == 2 => { - let properties:serde_json::Value = serde_json::from_str(¶ms[1])?; + DaoType::DeleteNamespaceByNamespace + | DaoType::DeletePartitionInfoByTableId + | DaoType::DeleteDataCommitInfoByTableId + | DaoType::DeleteTableNameIdByTableId + | DaoType::DeleteTablePathIdByTableId + | DaoType::DeleteTablePathIdByTablePath + if params.len() == 1 => + { + client.execute(&statement, &[¶ms[0]]).await + } + DaoType::DeleteTableInfoByIdAndPath + | DaoType::DeleteTableNameIdByTableNameAndNamespace + | DaoType::DeletePartitionInfoByTableIdAndPartitionDesc + | DaoType::DeleteDataCommitInfoByTableIdAndPartitionDesc + if params.len() == 2 => + { + client.execute(&statement, &[¶ms[0], ¶ms[1]]).await + } + DaoType::UpdateTableInfoPropertiesById | DaoType::UpdateNamespacePropertiesByNamespace if params.len() == 2 => { + let properties: serde_json::Value = serde_json::from_str(¶ms[1])?; client.execute(&statement, &[¶ms[0], &properties]).await } DaoType::DeletePreviousVersionPartition if params.len() == 3 => { @@ -1104,11 +1145,11 @@ pub async fn execute_update( client.execute(&statement, &[¶ms[0], ¶ms[1], &ts]).await } DaoType::DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId if params.len() == 3 => { - let commit_id:uuid::Uuid = uuid::Uuid::from_str(¶ms[2])?; + let commit_id: uuid::Uuid = uuid::Uuid::from_str(¶ms[2])?; client.execute(&statement, &[¶ms[0], ¶ms[1], &commit_id]).await } DaoType::UpdateTableInfoById if params.len() == 4 => { - let mut statement = "update table_info set ".to_owned(); + let mut statement = "update table_info set ".to_owned(); let mut idx = 2; let mut filter_params = Vec::::with_capacity(3); if !params[1].is_empty() { @@ -1117,13 +1158,17 @@ pub async fn execute_update( filter_params.push(params[1].clone()); } if !params[2].is_empty() { - if idx > 2 {statement += ",";} + if idx > 2 { + statement += ","; + } statement += format!("table_path = ${}::TEXT ", idx).as_str(); idx += 1; filter_params.push(params[2].clone()); } if !params[3].is_empty() { - if idx > 2 {statement += ",";} + if idx > 2 { + statement += ","; + } statement += format!("table_schema = ${}::TEXT ", idx).as_str(); idx += 1; filter_params.push(params[3].clone()); @@ -1131,8 +1176,19 @@ pub async fn execute_update( statement += " where table_id = $1::TEXT"; match idx { 3 => client.execute(&statement, &[¶ms[0], &filter_params[0]]).await, - 4 => client.execute(&statement, &[¶ms[0], &filter_params[0], &filter_params[1]]).await, - 5 => client.execute(&statement, &[¶ms[0], &filter_params[0], &filter_params[1], &filter_params[2]]).await, + 4 => { + client + .execute(&statement, &[¶ms[0], &filter_params[0], &filter_params[1]]) + .await + } + 5 => { + client + .execute( + &statement, + &[¶ms[0], &filter_params[0], &filter_params[1], &filter_params[2]], + ) + .await + } _ => todo!(), } } @@ -1140,14 +1196,14 @@ pub async fn execute_update( let concated_uuid = ¶ms[2]; if concated_uuid.len() % 32 != 0 { eprintln!("Invalid params of update_type={:?}, params={:?}", update_type, params); - return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); + return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } let uuid_num = concated_uuid.len() / 32; let mut uuid_list = Vec::::with_capacity(uuid_num); let mut idx = 0; for _ in 0..uuid_num { - let high = u64::from_str_radix(&concated_uuid[idx..idx+16], 16)?; - let low = u64::from_str_radix(&concated_uuid[idx+16..idx+32], 16)?; + let high = u64::from_str_radix(&concated_uuid[idx..idx + 16], 16)?; + let low = u64::from_str_radix(&concated_uuid[idx + 16..idx + 32], 16)?; uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string()); idx += 32; } @@ -1156,14 +1212,16 @@ pub async fn execute_update( let statement = format!( "delete from data_commit_info - where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id in ({}) ", uuid_str_list); + where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id in ({}) ", + uuid_str_list + ); let statement = client.prepare(&statement).await?; client.execute(&statement, &[¶ms[0], ¶ms[1]]).await } _ => { eprintln!("InvalidInput of type={:?}: {:?}", update_type, params); - return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)) + return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } }; match result { @@ -1175,12 +1233,12 @@ pub async fn execute_update( pub async fn execute_query_scalar( client: &mut Client, prepared: &mut PreparedStatementMap, - query_type: i32, - joined_string: String, + query_type: i32, + joined_string: String, ) -> Result, LakeSoulMetaDataError> { - if !(DAO_TYPE_QUERY_SCALAR_OFFSET..DAO_TYPE_UPDATE_OFFSET).contains(&query_type){ + if !(DAO_TYPE_QUERY_SCALAR_OFFSET..DAO_TYPE_UPDATE_OFFSET).contains(&query_type) { eprintln!("Invalid update_scalar_type_index: {:?}", query_type); - return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)) + return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } let query_type = DaoType::try_from(query_type).unwrap(); let statement = get_prepared_statement(client, prepared, &query_type).await?; @@ -1189,7 +1247,7 @@ pub async fn execute_query_scalar( .split(PARAM_DELIM) .collect::>() .iter() - .map(|str|str.to_string()) + .map(|str| str.to_string()) .collect::>(); match query_type { @@ -1200,52 +1258,54 @@ pub async fn execute_query_scalar( let ts = row.get::<_, Option>(0); match ts { Some(ts) => Ok(Some(format!("{}", ts))), - None => Ok(None) + None => Ok(None), } } - Err(e) => Err(LakeSoulMetaDataError::from(e)), + Err(e) => Err(LakeSoulMetaDataError::from(e)), Ok(None) => Ok(None), } } DaoType::GetLatestTimestampFromPartitionInfo if params.len() == 2 => { let result = client.query_opt(&statement, &[¶ms[0], ¶ms[1]]).await; match result { - Ok(Some(row)) => Ok(Some(format!("{}",row.get::<_, i64>(0)))), + Ok(Some(row)) => Ok(Some(format!("{}", row.get::<_, i64>(0)))), Ok(None) => Ok(None), - Err(e) => Err(LakeSoulMetaDataError::from(e)) + Err(e) => Err(LakeSoulMetaDataError::from(e)), } } DaoType::GetLatestVersionUpToTimeFromPartitionInfo if params.len() == 3 => { - let result = - client.query_opt(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2])?]).await; + let result = client + .query_opt(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2])?]) + .await; match result { Ok(Some(row)) => { let ts = row.get::<_, Option>(0); match ts { Some(ts) => Ok(Some(format!("{}", ts))), - None => Ok(None) + None => Ok(None), } } - Err(e) => Err(LakeSoulMetaDataError::from(e)), + Err(e) => Err(LakeSoulMetaDataError::from(e)), Ok(None) => Ok(None), } } DaoType::GetLatestVersionTimestampUpToTimeFromPartitionInfo if params.len() == 3 => { - let result = - client.query_opt(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2])?]).await; + let result = client + .query_opt(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2])?]) + .await; match result { Ok(Some(row)) => { let ts = row.get::<_, Option>(0); match ts { Some(ts) => Ok(Some(format!("{}", ts))), - None => Ok(None) + None => Ok(None), } } - Err(e) => Err(LakeSoulMetaDataError::from(e)), + Err(e) => Err(LakeSoulMetaDataError::from(e)), Ok(None) => Ok(None), } } - + _ => { eprintln!("InvalidInput of type={:?}: {:?}", query_type, params); Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)) @@ -1253,32 +1313,30 @@ pub async fn execute_query_scalar( } } -pub async fn clean_meta_for_test( - client: &Client -) ->Result { - let result = - client.batch_execute("delete from namespace; +pub async fn clean_meta_for_test(client: &Client) -> Result { + let result = client + .batch_execute( + "delete from namespace; delete from data_commit_info; delete from table_info; delete from table_path_id; delete from table_name_id; - delete from partition_info;").await; + delete from partition_info;", + ) + .await; match result { Ok(_) => Ok(0i32), Err(e) => Err(LakeSoulMetaDataError::from(e)), } } -pub async fn create_connection( - config: String -) -> Result { - let (client, connection) = - match tokio_postgres::connect(config.as_str(), NoTls).await { - Ok((client, connection))=>(client, connection), - Err(e)=>{ - eprintln!("{}", e); - return Err(LakeSoulMetaDataError::from(ErrorKind::ConnectionRefused)) - } +pub async fn create_connection(config: String) -> Result { + let (client, connection) = match tokio_postgres::connect(config.as_str(), NoTls).await { + Ok((client, connection)) => (client, connection), + Err(e) => { + eprintln!("{}", e); + return Err(LakeSoulMetaDataError::from(ErrorKind::ConnectionRefused)); + } }; spawn(async move { @@ -1287,21 +1345,20 @@ pub async fn create_connection( } }); - Ok( client ) + Ok(client) } - #[cfg(test)] mod tests { - use proto::proto::entity; use prost::Message; + use proto::proto::entity; #[tokio::test] async fn test_entity() -> std::io::Result<()> { let namespace = entity::Namespace { - namespace:"default".to_owned(), - properties:"{}".to_owned(), - comment:"".to_owned(), - domain:"public".to_owned(), + namespace: "default".to_owned(), + properties: "{}".to_owned(), + comment: "".to_owned(), + domain: "public".to_owned(), }; println!("{:?}", namespace); println!("{:?}", entity::Namespace::default()); @@ -1314,7 +1371,7 @@ mod tests { table_schema: "StructType {}".to_owned(), properties: "{}".to_owned(), partitions: "".to_owned(), - domain:"public".to_owned(), + domain: "public".to_owned(), }; println!("{:?}", table_info); println!("{:?}", table_info.encode_to_vec()); @@ -1322,11 +1379,10 @@ mod tests { println!("{:?}", table_info.encode_length_delimited_to_vec().len()); println!("{:?}", entity::TableInfo::default()); - let meta_info = entity::MetaInfo { list_partition: vec![], table_info: core::option::Option::None, - read_partition_info: vec![] + read_partition_info: vec![], }; println!("{:?}", meta_info); println!("{:?}", entity::MetaInfo::default()); @@ -1340,7 +1396,6 @@ mod tests { wrapper.namespace = vec![namespace]; println!("{:?}", wrapper.namespace); - Ok(()) } -} \ No newline at end of file +} diff --git a/rust/lakesoul-metadata/src/metadata_client.rs b/rust/lakesoul-metadata/src/metadata_client.rs index 427e9ab7b..ad416431c 100644 --- a/rust/lakesoul-metadata/src/metadata_client.rs +++ b/rust/lakesoul-metadata/src/metadata_client.rs @@ -4,17 +4,22 @@ use std::ops::DerefMut; use std::sync::Arc; -use std::{collections::HashMap, vec, env, fs}; +use std::{collections::HashMap, env, fs, vec}; -use proto::proto::entity::{TablePathId, TableNameId, TableInfo, PartitionInfo, JniWrapper, DataCommitInfo, MetaInfo, CommitOp, self}; use prost::Message; +use proto::proto::entity::{ + self, CommitOp, DataCommitInfo, JniWrapper, MetaInfo, PartitionInfo, TableInfo, TableNameId, TablePathId, +}; use tokio::sync::Mutex; use tokio_postgres::Client; use url::Url; -use crate::{execute_insert, PreparedStatementMap, DaoType, create_connection, clean_meta_for_test, execute_query, PARAM_DELIM, PARTITION_DESC_DELIM}; use crate::error::Result; +use crate::{ + clean_meta_for_test, create_connection, execute_insert, execute_query, DaoType, PreparedStatementMap, PARAM_DELIM, + PARTITION_DESC_DELIM, +}; pub struct MetaDataClient { client: Arc>, @@ -28,26 +33,36 @@ impl MetaDataClient { pub async fn from_env() -> Result { match env::var("lakesoul_home") { Ok(config_path) => { - let config = fs::read_to_string(&config_path).unwrap_or_else(|_| panic!("Fails at reading config file {}", &config_path)); - let config_map = config.split('\n').filter_map(|property| { - property.find('=').map(|idx| property.split_at(idx + 1)) - }).collect::>(); - let url = Url::parse(&config_map.get("lakesoul.pg.url=").unwrap_or(&"jdbc:postgresql://127.0.0.1:5432/lakesoul_test?stringtype=unspecified")[5..]).unwrap(); + let config = fs::read_to_string(&config_path) + .unwrap_or_else(|_| panic!("Fails at reading config file {}", &config_path)); + let config_map = config + .split('\n') + .filter_map(|property| property.find('=').map(|idx| property.split_at(idx + 1))) + .collect::>(); + let url = Url::parse( + &config_map + .get("lakesoul.pg.url=") + .unwrap_or(&"jdbc:postgresql://127.0.0.1:5432/lakesoul_test?stringtype=unspecified")[5..], + ) + .unwrap(); + Self::from_config(format!( + "host={} port={} dbname={} user={} password={}", + url.host_str().unwrap(), + url.port().unwrap(), + url.path_segments().unwrap().next().unwrap(), + config_map.get("lakesoul.pg.username=").unwrap_or(&"lakesoul_test"), + config_map.get("lakesoul.pg.password=").unwrap_or(&"lakesoul_test") + )) + .await + } + Err(_) => { Self::from_config( - format!( - "host={} port={} dbname={} user={} password={}", - url.host_str().unwrap(), - url.port().unwrap(), - url.path_segments().unwrap().next().unwrap(), - config_map.get("lakesoul.pg.username=").unwrap_or(&"lakesoul_test"), - config_map.get("lakesoul.pg.password=").unwrap_or(&"lakesoul_test")) - ).await + "host=127.0.0.1 port=5432 dbname=lakesoul_test user=lakesoul_test password=lakesoul_test" + .to_string(), + ) + .await } - Err(_) => Self::from_config( - "host=127.0.0.1 port=5432 dbname=lakesoul_test user=lakesoul_test password=lakesoul_test".to_string() - ).await } - } pub async fn from_config(config: String) -> Result { @@ -60,28 +75,32 @@ impl MetaDataClient { Ok(Self { client, prepared, - max_retry + max_retry, }) } - pub async fn create_table( - &self, - table_info: TableInfo - ) -> Result<()> { - self.insert_table_path_id(&table_path_id_from_table_info(&table_info)).await?; - self.insert_table_name_id(&table_name_id_from_table_info(&table_info)).await?; + pub async fn create_table(&self, table_info: TableInfo) -> Result<()> { + self.insert_table_path_id(&table_path_id_from_table_info(&table_info)) + .await?; + self.insert_table_name_id(&table_name_id_from_table_info(&table_info)) + .await?; self.insert_table_info(&table_info).await?; Ok(()) } async fn execute_insert(&self, insert_type: i32, wrapper: JniWrapper) -> Result { for times in 0..self.max_retry { - match execute_insert(self.client.lock().await.deref_mut(), self.prepared.lock().await.deref_mut(), insert_type, wrapper.clone()).await { + match execute_insert( + self.client.lock().await.deref_mut(), + self.prepared.lock().await.deref_mut(), + insert_type, + wrapper.clone(), + ) + .await + { Ok(count) => return Ok(count), - Err(_) if times < self.max_retry => { - continue - } - Err(e) => return Err(e) + Err(_) if times < self.max_retry => continue, + Err(e) => return Err(e), }; } Ok(0) @@ -89,35 +108,75 @@ impl MetaDataClient { async fn execute_query(&self, query_type: i32, joined_string: String) -> Result { for times in 0..self.max_retry { - match execute_query( self.client.lock().await.deref_mut(), self.prepared.lock().await.deref_mut(), query_type, joined_string.clone()).await { + match execute_query( + self.client.lock().await.deref_mut(), + self.prepared.lock().await.deref_mut(), + query_type, + joined_string.clone(), + ) + .await + { Ok(encoded) => return Ok(JniWrapper::decode(prost::bytes::Bytes::from(encoded))?), - Err(_) if times < self.max_retry => { - continue - } - Err(e) => return Err(e) + Err(_) if times < self.max_retry => continue, + Err(e) => return Err(e), }; } Ok(Default::default()) } async fn insert_table_info(&self, table_info: &TableInfo) -> Result { - self.execute_insert(DaoType::InsertTableInfo as i32, JniWrapper{table_info: vec![table_info.clone()], ..Default::default()}).await + self.execute_insert( + DaoType::InsertTableInfo as i32, + JniWrapper { + table_info: vec![table_info.clone()], + ..Default::default() + }, + ) + .await } - async fn insert_table_name_id(&self, table_name_id: &TableNameId) -> Result{ - self.execute_insert(DaoType::InsertTableNameId as i32, JniWrapper{table_name_id: vec![table_name_id.clone()], ..Default::default()}).await + async fn insert_table_name_id(&self, table_name_id: &TableNameId) -> Result { + self.execute_insert( + DaoType::InsertTableNameId as i32, + JniWrapper { + table_name_id: vec![table_name_id.clone()], + ..Default::default() + }, + ) + .await } - async fn insert_table_path_id(&self, table_path_id: &TablePathId) -> Result{ - self.execute_insert(DaoType::InsertTablePathId as i32, JniWrapper{table_path_id: vec![table_path_id.clone()], ..Default::default()}).await + async fn insert_table_path_id(&self, table_path_id: &TablePathId) -> Result { + self.execute_insert( + DaoType::InsertTablePathId as i32, + JniWrapper { + table_path_id: vec![table_path_id.clone()], + ..Default::default() + }, + ) + .await } async fn insert_data_commit_info(&self, data_commit_info: &DataCommitInfo) -> Result { - self.execute_insert(DaoType::InsertDataCommitInfo as i32, JniWrapper{data_commit_info: vec![data_commit_info.clone()], ..Default::default()}).await + self.execute_insert( + DaoType::InsertDataCommitInfo as i32, + JniWrapper { + data_commit_info: vec![data_commit_info.clone()], + ..Default::default() + }, + ) + .await } async fn transaction_insert_partition_info(&self, partition_info_list: Vec) -> Result { - self.execute_insert(DaoType::TransactionInsertPartitionInfo as i32, JniWrapper { partition_info: partition_info_list, ..Default::default()}).await + self.execute_insert( + DaoType::TransactionInsertPartitionInfo as i32, + JniWrapper { + partition_info: partition_info_list, + ..Default::default() + }, + ) + .await } pub async fn meta_cleanup(&self) -> Result { @@ -128,34 +187,37 @@ impl MetaDataClient { let table_info = meta_info.table_info.unwrap(); if !table_info.table_name.is_empty() { // todo: updateTableShortName - - } // todo: updateTableProperties // conflict handling - let _raw_map = meta_info.list_partition + let _raw_map = meta_info + .list_partition .iter() .map(|partition_info| (partition_info.partition_desc.clone(), partition_info.clone())) .collect::>(); - let partition_desc_list = meta_info.list_partition + let partition_desc_list = meta_info + .list_partition .iter() .map(|partition_info| partition_info.partition_desc.clone()) .collect::>(); - let _snapshot_list = meta_info.list_partition + let _snapshot_list = meta_info + .list_partition .iter() .flat_map(|partition_info| partition_info.snapshot.clone()) .collect::>(); // conflict handling - let cur_map = self.get_cur_partition_map(&table_info.table_id, &partition_desc_list).await?; - + let cur_map = self + .get_cur_partition_map(&table_info.table_id, &partition_desc_list) + .await?; match commit_op { CommitOp::AppendCommit | CommitOp::MergeCommit => { - let new_partition_list = meta_info.list_partition + let new_partition_list = meta_info + .list_partition .iter() .map(|partition_info| { let partition_desc = &partition_info.partition_desc; @@ -163,7 +225,9 @@ impl MetaDataClient { Some(cur_partition_info) => { let mut cur_partition_info = cur_partition_info.clone(); cur_partition_info.domain = self.get_table_domain(&table_info.table_id).unwrap(); - cur_partition_info.snapshot.extend_from_slice(&partition_info.snapshot[..]); + cur_partition_info + .snapshot + .extend_from_slice(&partition_info.snapshot[..]); cur_partition_info.version += 1; cur_partition_info.commit_op = commit_op as i32; cur_partition_info.expression = partition_info.expression.clone(); @@ -178,7 +242,7 @@ impl MetaDataClient { commit_op: commit_op as i32, expression: partition_info.expression.clone(), ..Default::default() - } + }, } }) .collect::>(); @@ -191,12 +255,17 @@ impl MetaDataClient { } } - async fn get_cur_partition_map(&self, table_id: &str, partition_desc_list: &[String]) -> Result> { - Ok(self.get_partition_info_by_table_id_and_partition_list(table_id, partition_desc_list).await? + async fn get_cur_partition_map( + &self, + table_id: &str, + partition_desc_list: &[String], + ) -> Result> { + Ok(self + .get_partition_info_by_table_id_and_partition_list(table_id, partition_desc_list) + .await? .iter() - .map(|partition_info|(partition_info.partition_desc.clone(), partition_info.clone())) - .collect() - ) + .map(|partition_info| (partition_info.partition_desc.clone(), partition_info.clone())) + .collect()) } pub async fn commit_data_commit_info(&self, data_commit_info: DataCommitInfo) -> Result<()> { @@ -205,7 +274,10 @@ impl MetaDataClient { let commit_op = data_commit_info.commit_op; let commit_id = &data_commit_info.commit_id.clone().unwrap(); let commit_id_str = uuid::Uuid::from_u64_pair(commit_id.high, commit_id.low).to_string(); - match self.get_single_data_commit_info(table_id, partition_desc, &commit_id_str).await? { + match self + .get_single_data_commit_info(table_id, partition_desc, &commit_id_str) + .await? + { Some(data_commit_info) if data_commit_info.committed => { return Ok(()); } @@ -216,18 +288,22 @@ impl MetaDataClient { }; let table_info = Some(self.get_table_info_by_table_id(table_id).await?); let domain = self.get_table_domain(table_id)?; - self.commit_data(MetaInfo { - table_info, - list_partition: vec![PartitionInfo { - table_id: table_id.clone(), - partition_desc: partition_desc.clone(), - commit_op, - domain, - snapshot: vec![commit_id.clone()], + self.commit_data( + MetaInfo { + table_info, + list_partition: vec![PartitionInfo { + table_id: table_id.clone(), + partition_desc: partition_desc.clone(), + commit_op, + domain, + snapshot: vec![commit_id.clone()], + ..Default::default() + }], ..Default::default() - }], - ..Default::default() - }, CommitOp::from_i32(commit_op).unwrap()).await + }, + CommitOp::from_i32(commit_op).unwrap(), + ) + .await } pub fn get_table_domain(&self, _table_id: &str) -> Result { @@ -236,38 +312,63 @@ impl MetaDataClient { } pub async fn get_table_name_id_by_table_name(&self, table_name: &str, namespace: &str) -> Result { - match self.execute_query(DaoType::SelectTableNameIdByTableName as i32, [table_name, namespace].join(PARAM_DELIM)).await { + match self + .execute_query( + DaoType::SelectTableNameIdByTableName as i32, + [table_name, namespace].join(PARAM_DELIM), + ) + .await + { Ok(wrapper) => Ok(wrapper.table_name_id[0].clone()), - Err(err) => Err(err) + Err(err) => Err(err), } } pub async fn get_table_info_by_table_name(&self, table_name: &str, namespace: &str) -> Result { - match self.execute_query(DaoType::SelectTableInfoByTableNameAndNameSpace as i32, [table_name, namespace].join(PARAM_DELIM)).await { - Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::Internal(format!("Table '{}' not found", table_name))), + match self + .execute_query( + DaoType::SelectTableInfoByTableNameAndNameSpace as i32, + [table_name, namespace].join(PARAM_DELIM), + ) + .await + { + Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::Internal( + format!("Table '{}' not found", table_name), + )), Ok(wrapper) => Ok(wrapper.table_info[0].clone()), - Err(err) => Err(err) + Err(err) => Err(err), } } pub async fn get_table_info_by_table_path(&self, table_path: &str) -> Result { - match self.execute_query(DaoType::SelectTablePathIdByTablePath as i32, table_path.to_string()).await { - Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::Internal(format!("Table '{}' not found", table_path))), + match self + .execute_query(DaoType::SelectTablePathIdByTablePath as i32, table_path.to_string()) + .await + { + Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::Internal( + format!("Table '{}' not found", table_path), + )), Ok(wrapper) => Ok(wrapper.table_info[0].clone()), - Err(err) => Err(err) + Err(err) => Err(err), } } - pub async fn get_table_info_by_table_id(&self, table_id: &str) -> Result { - match self.execute_query(DaoType::SelectTableInfoByTableId as i32, table_id.to_string()).await { + match self + .execute_query(DaoType::SelectTableInfoByTableId as i32, table_id.to_string()) + .await + { Ok(wrapper) => Ok(wrapper.table_info[0].clone()), - Err(err) => Err(err) + Err(err) => Err(err), } } - - pub async fn get_data_files_by_table_name(&self, table_name: &str, partitions: Vec<(&str, &str)>, namespace: &str) -> Result> { + pub async fn get_data_files_by_table_name( + &self, + table_name: &str, + partitions: Vec<(&str, &str)>, + namespace: &str, + ) -> Result> { let partition_filter = partitions .iter() .map(|(k, v)| format!("{}={}", k, v)) @@ -286,7 +387,8 @@ impl MetaDataClient { let _data_file_list = _data_commit_info_list .iter() .flat_map(|data_commit_info| { - data_commit_info.file_ops + data_commit_info + .file_ops .iter() .map(|file_op| file_op.path.clone()) .collect::>() @@ -298,16 +400,26 @@ impl MetaDataClient { Ok(data_commit_info_list) } - async fn get_data_commit_info_of_single_partition(&self, partition_info: &PartitionInfo) -> Result> { + async fn get_data_commit_info_of_single_partition( + &self, + partition_info: &PartitionInfo, + ) -> Result> { let table_id = &partition_info.table_id; let partition_desc = &partition_info.partition_desc; - let joined_commit_id = &partition_info.snapshot + let joined_commit_id = &partition_info + .snapshot .iter() .map(|commit_id| format!("{:0>16x}{:0>16x}", commit_id.high, commit_id.low)) .collect::>() .join(""); let joined_string = [table_id.as_str(), partition_desc.as_str(), joined_commit_id.as_str()].join(PARAM_DELIM); - match self.execute_query(DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList as i32, joined_string).await { + match self + .execute_query( + DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList as i32, + joined_string, + ) + .await + { Ok(wrapper) => Ok(wrapper.data_commit_info), Err(e) => Err(e), } @@ -319,14 +431,28 @@ impl MetaDataClient { } pub async fn get_all_partition_info(&self, table_id: &str) -> Result> { - match self.execute_query(DaoType::ListPartitionByTableId as i32, table_id.to_string()).await { + match self + .execute_query(DaoType::ListPartitionByTableId as i32, table_id.to_string()) + .await + { Ok(wrapper) => Ok(wrapper.partition_info), Err(e) => Err(e), } } - pub async fn get_single_data_commit_info(&self, table_id: &str, partition_desc: &str, commit_id: &str) -> Result> { - match self.execute_query(DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId as i32, [table_id, partition_desc, commit_id].join(PARAM_DELIM)).await { + pub async fn get_single_data_commit_info( + &self, + table_id: &str, + partition_desc: &str, + commit_id: &str, + ) -> Result> { + match self + .execute_query( + DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId as i32, + [table_id, partition_desc, commit_id].join(PARAM_DELIM), + ) + .await + { Ok(wrapper) => Ok(if wrapper.data_commit_info.is_empty() { None } else { @@ -336,30 +462,37 @@ impl MetaDataClient { } } - pub async fn get_partition_info_by_table_id_and_partition_list(&self, table_id: &str, partition_desc_list: &[String]) -> Result> { - match self.execute_query(DaoType::ListPartitionDescByTableIdAndParList as i32, [table_id, partition_desc_list.join(PARTITION_DESC_DELIM).as_str()].join(PARAM_DELIM)).await { + pub async fn get_partition_info_by_table_id_and_partition_list( + &self, + table_id: &str, + partition_desc_list: &[String], + ) -> Result> { + match self + .execute_query( + DaoType::ListPartitionDescByTableIdAndParList as i32, + [table_id, partition_desc_list.join(PARTITION_DESC_DELIM).as_str()].join(PARAM_DELIM), + ) + .await + { Ok(wrapper) => Ok(wrapper.partition_info), Err(e) => Err(e), } - } - } pub fn table_path_id_from_table_info(table_info: &TableInfo) -> TablePathId { - TablePathId { - table_path: table_info.table_path.clone(), - table_id: table_info.table_id.clone(), - table_namespace: table_info.table_namespace.clone(), - domain: table_info.domain.clone() + TablePathId { + table_path: table_info.table_path.clone(), + table_id: table_info.table_id.clone(), + table_namespace: table_info.table_namespace.clone(), + domain: table_info.domain.clone(), } } pub fn table_name_id_from_table_info(table_info: &TableInfo) -> TableNameId { - TableNameId { - table_name: table_info.table_name.clone(), - table_id: table_info.table_id.clone(), - table_namespace: table_info.table_namespace.clone(), - domain: table_info.domain.clone() + TableNameId { + table_name: table_info.table_name.clone(), + table_id: table_info.table_id.clone(), + table_namespace: table_info.table_namespace.clone(), + domain: table_info.domain.clone(), } } - diff --git a/rust/proto/build.rs b/rust/proto/build.rs index d865591df..58fe3b65a 100644 --- a/rust/proto/build.rs +++ b/rust/proto/build.rs @@ -5,10 +5,9 @@ extern crate prost_build; fn main() { - #[cfg(target_os = "linux")] + #[cfg(target_os = "linux")] { std::env::set_var("PROTOC", protobuf_src::protoc()); } - prost_build::compile_protos(&["src/entity.proto"], - &["src/"]).unwrap(); -} \ No newline at end of file + prost_build::compile_protos(&["src/entity.proto"], &["src/"]).unwrap(); +} diff --git a/rust/proto/src/lib.rs b/rust/proto/src/lib.rs index 800899bfc..9721eadcb 100644 --- a/rust/proto/src/lib.rs +++ b/rust/proto/src/lib.rs @@ -2,8 +2,8 @@ // // SPDX-License-Identifier: Apache-2.0 -pub mod proto{ +pub mod proto { pub mod entity { include!(concat!(env!("OUT_DIR"), "/proto.entity.rs")); } -} \ No newline at end of file +} diff --git a/script/benchmark/presto/catalog/lakesoul.properties b/script/benchmark/presto/catalog/lakesoul.properties index 27885b5e4..d6c49dcc8 100644 --- a/script/benchmark/presto/catalog/lakesoul.properties +++ b/script/benchmark/presto/catalog/lakesoul.properties @@ -7,6 +7,7 @@ fs.s3a.access.key=minioadmin1 fs.s3a.secret.key=minioadmin1 fs.s3a.bucket=lakesoul-test-bucket fs.s3a.endpoint=http://minio:9000 +fs.s3a.path.style.access=true