diff --git a/Cargo.lock b/Cargo.lock index 8d86023bb0518..85492f16b8881 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -17,12 +17,6 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" -[[package]] -name = "ahash" -version = "0.4.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "739f4a8db6605981345c5654f3a85b056ce52f37a39d34da03f25bf2151ea16e" - [[package]] name = "ahash" version = "0.6.3" @@ -56,9 +50,9 @@ dependencies = [ [[package]] name = "alloc-no-stdlib" -version = "2.0.1" +version = "2.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5192ec435945d87bc2f70992b4d818154b5feede43c09fb7592146374eac90a6" +checksum = "35ef4730490ad1c4eae5c4325b2a95f521d023e5c885853ff7aca0a6a1631db3" [[package]] name = "alloc-stdlib" @@ -99,6 +93,12 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "237430fd6ed3740afe94eefcc278ae21e050285be882804e0d6e8695f0c94691" +[[package]] +name = "arrayref" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" + [[package]] name = "arrayvec" version = "0.4.12" @@ -115,43 +115,48 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "23b62fc65de8e4e7f52534fb52b0f3ed04746ae267519eef2a83941e8085068b" [[package]] -name = "arrow" -version = "5.0.0-SNAPSHOT" -source = "git+https://github.com/datafuse-extras/arrow-rs?rev=86d127b#86d127ba93595f169c80e966e75daf402497d541" +name = "arrow-flight" +version = "0.1.0" +source = "git+https://github.com/datafuse-extras/arrow2?rev=ea2f82d#ea2f82d35fc4973117873d814e83831def1ccaff" dependencies = [ - "bitflags", + "arrow2", + "bytes", + "futures", + "prost 0.7.0", + "prost-derive 0.7.0", + "tokio", + "tonic", +] + +[[package]] +name = "arrow2" +version = "0.2.0" +source = "git+https://github.com/datafuse-extras/arrow2?rev=ea2f82d#ea2f82d35fc4973117873d814e83831def1ccaff" +dependencies = [ + "ahash 0.7.4", + "base64", "chrono", "csv", "flatbuffers", - "getrandom 0.2.3", + "futures", + "hash_hasher", "hex", "indexmap", + "itertools 0.10.1", "lazy_static", "lexical-core", + "lz4", "multiversion", "num", "packed_simd_2", + "parquet2 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)", + "prettytable-rs", "rand 0.8.4", "regex", "serde", "serde_derive", "serde_json", -] - -[[package]] -name = "arrow-flight" -version = "5.0.0-SNAPSHOT" -source = "git+https://github.com/datafuse-extras/arrow-rs?rev=86d127b#86d127ba93595f169c80e966e75daf402497d541" -dependencies = [ - "arrow", - "base64", - "bytes", - "proc-macro2", - "prost 0.7.0", - "prost-derive 0.7.0", - "tokio", - "tonic", - "tonic-build", + "zstd", ] [[package]] @@ -311,6 +316,15 @@ version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cf1de2fe8c75bc145a2f577add951f8134889b4795d47466a54a5c846d691693" +[[package]] +name = "bitpacking" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8c7d2ac73c167c06af4a5f37e6e59d84148d57ccbe4480b76f0273eefea82d7" +dependencies = [ + "crunchy", +] + [[package]] name = "bitvec" version = "0.22.3" @@ -323,6 +337,17 @@ dependencies = [ "wyz", ] +[[package]] +name = "blake2b_simd" +version = "0.5.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "afa748e348ad3be8263be728124b24a24f268266f6f5d58af9d75f6a40b5c587" +dependencies = [ + "arrayref", + "arrayvec 0.5.2", + "constant_time_eq", +] + [[package]] name = "block-buffer" version = "0.9.0" @@ -334,9 +359,9 @@ dependencies = [ [[package]] name = "brotli" -version = "3.3.0" +version = "3.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f29919120f08613aadcd4383764e00526fc9f18b6c0895814faeed0dd78613e" +checksum = "71cb90ade945043d3d53597b2fc359bb063db8ade2bcffe7997351d0756e9d50" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -345,9 +370,9 @@ dependencies = [ [[package]] name = "brotli-decompressor" -version = "2.3.1" +version = "2.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1052e1c3b8d4d80eb84a8b94f0a1498797b5fb96314c001156a1c761940ef4ec" +checksum = "59ad2d4653bf5ca36ae797b1f4bb4dbddb60ce49ca4aed8a2ce4829f60425b80" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -670,9 +695,9 @@ dependencies = [ name = "common-arrow" version = "0.1.0" dependencies = [ - "arrow", "arrow-flight", - "parquet", + "arrow2", + "parquet2 0.1.0 (git+https://github.com/datafuse-extras/parquet2?branch=main)", ] [[package]] @@ -949,6 +974,12 @@ version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f92cfa0fd5690b3cf8c1ef2cabbd9b7ef22fa53cf5e1f92b05103f6d5d1cf6e7" +[[package]] +name = "constant_time_eq" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "245097e9a4535ee1e3e3931fcfcd55a796a44c643e8596ff6566d68f09b87bbc" + [[package]] name = "core-foundation" version = "0.9.1" @@ -1036,7 +1067,7 @@ checksum = "69323bff1fb41c635347b8ead484a5ca6c3f11914d784170b158d8449ab07f8e" dependencies = [ "cfg-if 0.1.10", "crossbeam-channel 0.4.4", - "crossbeam-deque 0.7.3", + "crossbeam-deque 0.7.4", "crossbeam-epoch 0.8.2", "crossbeam-queue 0.2.3", "crossbeam-utils 0.7.2", @@ -1050,7 +1081,7 @@ checksum = "4ae5588f6b3c3cb05239e90bd110f257254aecd01e4635400391aeae07497845" dependencies = [ "cfg-if 1.0.0", "crossbeam-channel 0.5.1", - "crossbeam-deque 0.8.0", + "crossbeam-deque 0.8.1", "crossbeam-epoch 0.9.5", "crossbeam-queue 0.3.2", "crossbeam-utils 0.8.5", @@ -1078,9 +1109,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f02af974daeee82218205558e51ec8768b48cf524bd01d550abe5573a608285" +checksum = "c20ff29ded3204c5106278a81a38f4b482636ed4fa1e6cfbeef193291beb29ed" dependencies = [ "crossbeam-epoch 0.8.2", "crossbeam-utils 0.7.2", @@ -1089,9 +1120,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94af6efb46fef72616855b036a624cf27ba656ffc9be1b9a3c931cfc7749a9a9" +checksum = "6455c0ca19f0d2fbf751b908d5c55c1f5cbc65e03c4225427254b46890bdde1e" dependencies = [ "cfg-if 1.0.0", "crossbeam-epoch 0.9.5", @@ -1193,6 +1224,12 @@ dependencies = [ "winapi", ] +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + [[package]] name = "crypto-bigint" version = "0.2.2" @@ -1286,7 +1323,7 @@ dependencies = [ "ansi_term 0.12.1", "clap", "common-building", - "dirs", + "dirs 3.0.2", "dyn-clone", "flate2", "indicatif", @@ -1365,6 +1402,17 @@ dependencies = [ "generic-array", ] +[[package]] +name = "dirs" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fd78930633bd1c6e35c4b42b1df7b0cbc6bc191146e512bb3bedf243fcc3901" +dependencies = [ + "libc", + "redox_users 0.3.5", + "winapi", +] + [[package]] name = "dirs" version = "3.0.2" @@ -1391,7 +1439,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "03d86534ed367a67548dc68113a0f5db55432fdfbb6e6f9d77704397d95d5780" dependencies = [ "libc", - "redox_users", + "redox_users 0.4.0", "winapi", ] @@ -1402,7 +1450,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ebda144c4fe02d1f7ea1a7d9641b6fc6b580adcfa024ae48797ecdeb6825b4d" dependencies = [ "libc", - "redox_users", + "redox_users 0.4.0", "winapi", ] @@ -1590,7 +1638,7 @@ checksum = "1d34cfa13a63ae058bfa601fe9e313bbdb3746427c1459185464ce0fcf62e1e8" dependencies = [ "cfg-if 1.0.0", "libc", - "redox_syscall", + "redox_syscall 0.2.9", "winapi", ] @@ -2031,10 +2079,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7fcd999463524c52659517fe2cea98493cfe485d10565e7b0fb07dbba7ad2753" dependencies = [ "cfg-if 1.0.0", - "js-sys", "libc", "wasi 0.10.0+wasi-snapshot-preview1", - "wasm-bindgen", ] [[package]] @@ -2111,13 +2157,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "62aca2aba2d62b4a7f5b33f3712cb1b0692779a56fb510499d5c0aa594daeaf3" [[package]] -name = "hashbrown" -version = "0.9.1" +name = "hash_hasher" +version = "2.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7afe4a420e3fe79967a00898cc1f4db7c8a49a9333a29f8a4bd76a253d5cd04" -dependencies = [ - "ahash 0.4.7", -] +checksum = "74721d007512d0cb3338cd20f0654ac913920061a4c4d0d8708edb3f2a698c0c" [[package]] name = "hashbrown" @@ -2284,7 +2327,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.0", + "socket2 0.4.1", "tokio", "tower-service", "tracing", @@ -2309,7 +2352,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bc633605454125dec4b66843673f01c7df2b89479b32e0ed634e43a91cff62a5" dependencies = [ "autocfg 1.0.1", - "hashbrown 0.11.2", + "hashbrown", ] [[package]] @@ -2619,11 +2662,11 @@ dependencies = [ [[package]] name = "lru" -version = "0.6.5" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f374d42cdfc1d7dbf3d3dec28afab2eb97ffbf43a3234d795b5986dbf4b90ba" +checksum = "7ea2d928b485416e8908cff2d97d621db22b27f7b3b6729e438bcf42c671ba91" dependencies = [ - "hashbrown 0.9.1", + "hashbrown", ] [[package]] @@ -2784,7 +2827,7 @@ dependencies = [ "crossbeam-epoch 0.9.5", "crossbeam-utils 0.8.5", "dashmap", - "hashbrown 0.11.2", + "hashbrown", "indexmap", "metrics", "num_cpus", @@ -2945,7 +2988,7 @@ dependencies = [ "percent-encoding", "serde", "serde_json", - "socket2 0.4.0", + "socket2 0.4.1", "twox-hash", "url", ] @@ -3383,38 +3426,53 @@ dependencies = [ "cfg-if 1.0.0", "instant", "libc", - "redox_syscall", + "redox_syscall 0.2.9", "smallvec", "winapi", ] [[package]] -name = "parquet" -version = "5.0.0-SNAPSHOT" -source = "git+https://github.com/datafuse-extras/arrow-rs?rev=86d127b#86d127ba93595f169c80e966e75daf402497d541" +name = "parquet-format" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5bc6b23543b5dedc8f6cce50758a35e5582e148e0cfa26bd0cacd569cda5b71" dependencies = [ - "arrow", - "base64", + "thrift", +] + +[[package]] +name = "parquet2" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2cab3c2da3dc2e9e5e62fbf409d2c635ac5389ea9fdd5280e3660ae0f900e000" +dependencies = [ + "bitpacking", "brotli", - "byteorder", - "chrono", "flate2", + "futures", "lz4", - "num-bigint 0.4.0", "parquet-format", - "rand 0.8.4", "snap", + "streaming-iterator", "thrift", "zstd", ] [[package]] -name = "parquet-format" -version = "2.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5bc6b23543b5dedc8f6cce50758a35e5582e148e0cfa26bd0cacd569cda5b71" +name = "parquet2" +version = "0.1.0" +source = "git+https://github.com/datafuse-extras/parquet2?branch=main#20a1fb9d705c827e045504a5d5cb01537ba07c27" dependencies = [ + "bitpacking", + "brotli", + "flate2", + "futures", + "lz4", + "parquet-format", + "snap", + "streaming-iterator", "thrift", + "zstd", ] [[package]] @@ -3729,6 +3787,20 @@ dependencies = [ "output_vt100", ] +[[package]] +name = "prettytable-rs" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fd04b170004fa2daccf418a7f8253aaf033c27760b5f225889024cf66d7ac2e" +dependencies = [ + "atty", + "csv", + "encode_unicode", + "lazy_static", + "term", + "unicode-width", +] + [[package]] name = "proc-macro-error" version = "1.0.4" @@ -3767,9 +3839,9 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.27" +version = "1.0.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0d8caf72986c1a598726adc988bb5984792ef84f5ee5aa50209145ee8077038" +checksum = "5c7ed8b8c7b886ea3ed7dde405212185f423ab44682667c8c6dd14aa1d9f6612" dependencies = [ "unicode-xid", ] @@ -3809,7 +3881,7 @@ dependencies = [ "prost 0.7.0", "prost-types 0.7.0", "tempfile", - "which 4.1.0", + "which 4.2.2", ] [[package]] @@ -3827,7 +3899,7 @@ dependencies = [ "prost 0.8.0", "prost-types 0.8.0", "tempfile", - "which 4.1.0", + "which 4.2.2", ] [[package]] @@ -4045,7 +4117,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c06aca804d41dbc8ba42dfd964f0d01334eceb64314b9ecf7c5fad5188a06d90" dependencies = [ "autocfg 1.0.1", - "crossbeam-deque 0.8.0", + "crossbeam-deque 0.8.1", "either", "rayon-core", ] @@ -4057,12 +4129,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d78120e2c850279833f1dd3582f730c4ab53ed95aeaaaa862a2a5c71b1656d8e" dependencies = [ "crossbeam-channel 0.5.1", - "crossbeam-deque 0.8.0", + "crossbeam-deque 0.8.1", "crossbeam-utils 0.8.5", "lazy_static", "num_cpus", ] +[[package]] +name = "redox_syscall" +version = "0.1.57" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41cc0f7e4d5d4544e8861606a285bb08d3e70712ccc7d2b84d7c0ccfaf4b05ce" + [[package]] name = "redox_syscall" version = "0.2.9" @@ -4072,6 +4150,17 @@ dependencies = [ "bitflags", ] +[[package]] +name = "redox_users" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de0737333e7a9502c789a36d7c7fa6092a49895d4faa31ca5df163857ded2e9d" +dependencies = [ + "getrandom 0.1.16", + "redox_syscall 0.1.57", + "rust-argon2", +] + [[package]] name = "redox_users" version = "0.4.0" @@ -4079,7 +4168,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "528532f3d801c87aec9def2add9ca802fe569e44a544afe633765267840abe64" dependencies = [ "getrandom 0.2.3", - "redox_syscall", + "redox_syscall 0.2.9", ] [[package]] @@ -4180,6 +4269,18 @@ dependencies = [ "fsio", ] +[[package]] +name = "rust-argon2" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b18820d944b33caa75a71378964ac46f58517c92b6ae5f762636247c09e78fb" +dependencies = [ + "base64", + "blake2b_simd", + "constant_time_eq", + "crossbeam-utils 0.8.5", +] + [[package]] name = "rust_decimal" version = "1.15.0" @@ -4218,7 +4319,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver 1.0.3", + "semver 1.0.4", ] [[package]] @@ -4367,9 +4468,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f3aac57ee7f3272d8395c6e4f502f434f0e289fcd62876f70daa008c20dcabe" +checksum = "568a8e6258aa33c13358f81fd834adb854c6f7c9468520910a9b1e8fac068012" [[package]] name = "semver-parser" @@ -4474,9 +4575,9 @@ dependencies = [ [[package]] name = "sharded-slab" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79c719719ee05df97490f80a45acfc99e5a30ce98a1e4fb67aee422745ae14e3" +checksum = "ef79578eaf1846837727db975d587f15f6d8abd3900f4aa58e3fa6ae0fd9e4d1" dependencies = [ "lazy_static", ] @@ -4614,9 +4715,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e3dfc207c526015c632472a77be09cf1b6e46866581aecae5cc38fb4235dea2" +checksum = "765f090f0e423d2b55843402a07915add955e7d60657db13707a159727326cad" dependencies = [ "libc", "winapi", @@ -4727,6 +4828,12 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" +[[package]] +name = "streaming-iterator" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "303235c177994a476226b80d076bd333b7b560fb05bd242a10609d11b07f81f5" + [[package]] name = "strsim" version = "0.8.0" @@ -4921,11 +5028,22 @@ dependencies = [ "cfg-if 1.0.0", "libc", "rand 0.8.4", - "redox_syscall", + "redox_syscall 0.2.9", "remove_dir_all", "winapi", ] +[[package]] +name = "term" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edd106a334b7657c10b7c540a0106114feadeb4dc314513e97df481d5d966f42" +dependencies = [ + "byteorder", + "dirs 1.0.5", + "winapi", +] + [[package]] name = "termcolor" version = "1.1.2" @@ -5798,11 +5916,12 @@ dependencies = [ [[package]] name = "which" -version = "4.1.0" +version = "4.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b55551e42cbdf2ce2bedd2203d0cc08dba002c27510f86dab6d0ce304cba3dfe" +checksum = "ea187a8ef279bc014ec368c27a920da2024d2a711109bfbe3440585d5cf27ad9" dependencies = [ "either", + "lazy_static", "libc", ] @@ -5893,18 +6012,18 @@ dependencies = [ [[package]] name = "zstd" -version = "0.9.0+zstd.1.5.0" +version = "0.6.1+zstd.1.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07749a5dc2cb6b36661290245e350f15ec3bbb304e493db54a1d354480522ccd" +checksum = "5de55e77f798f205d8561b8fe2ef57abfb6e0ff2abe7fd3c089e119cdb5631a3" dependencies = [ "zstd-safe", ] [[package]] name = "zstd-safe" -version = "4.1.1+zstd.1.5.0" +version = "3.0.1+zstd.1.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c91c90f2c593b003603e5e0493c837088df4469da25aafff8bce42ba48caf079" +checksum = "1387cabcd938127b30ce78c4bf00b30387dddf704e3f0881dbc4ff62b5566f8c" dependencies = [ "libc", "zstd-sys", @@ -5912,9 +6031,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "1.6.1+zstd.1.5.0" +version = "1.4.20+zstd.1.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "615120c7a2431d16cf1cf979e7fc31ba7a5b5e5707b29c8a99e5dbf8a8392a33" +checksum = "ebd5b733d7cf2d9447e2c3e76a5589b4f5e5ae065c22a2bc0b023cbc331b6c8e" dependencies = [ "cc", "libc", diff --git a/common/arrow/Cargo.toml b/common/arrow/Cargo.toml index 7d78f0590998d..5437b2a94c61d 100644 --- a/common/arrow/Cargo.toml +++ b/common/arrow/Cargo.toml @@ -13,11 +13,9 @@ simd = ["arrow/simd"] # Workspace dependencies # Github dependencies - -arrow = { git="https://github.com/datafuse-extras/arrow-rs", rev = "86d127b" } -arrow-flight = {git="https://github.com/datafuse-extras/arrow-rs", rev = "86d127b" } -parquet = { git="https://github.com/datafuse-extras/arrow-rs", rev = "86d127b", features = ["arrow"] } - +arrow = { package = "arrow2", git="https://github.com/datafuse-extras/arrow2", rev = "ea2f82d" } +arrow-flight = { git="https://github.com/datafuse-extras/arrow2", rev = "ea2f82d" } +parquet = {package = "parquet2", git = "https://github.com/datafuse-extras/parquet2", branch = "main"} # Crates.io dependencies [dev-dependencies] diff --git a/common/datablocks/src/data_block.rs b/common/datablocks/src/data_block.rs index fb6dba4ea63b3..c28567ae02e4b 100644 --- a/common/datablocks/src/data_block.rs +++ b/common/datablocks/src/data_block.rs @@ -7,6 +7,7 @@ use std::fmt; use std::sync::Arc; use common_arrow::arrow; +use common_arrow::arrow::array::ArrayRef; use common_arrow::arrow::record_batch::RecordBatch; use common_datavalues::columns::DataColumn; use common_datavalues::series::IntoSeries; @@ -47,9 +48,9 @@ impl DataBlock { pub fn empty_with_schema(schema: DataSchemaRef) -> Self { let mut columns = vec![]; for f in schema.fields().iter() { - columns.push(DataColumn::Array( - arrow::array::new_empty_array(&f.data_type().to_arrow()).into_series(), - )) + let array = arrow::array::new_empty_array(f.data_type().to_arrow()); + let array: ArrayRef = Arc::from(array); + columns.push(DataColumn::Array(array.into_series())) } DataBlock { schema, columns } } @@ -152,7 +153,7 @@ impl TryFrom for DataBlock { type Error = ErrorCode; fn try_from(v: arrow::record_batch::RecordBatch) -> Result { - let schema = Arc::new(v.schema().into()); + let schema = Arc::new(v.schema().as_ref().into()); let series = v .columns() .iter() diff --git a/common/datablocks/src/kernels/data_block_slice.rs b/common/datablocks/src/kernels/data_block_slice.rs index d0330dd1ae7d5..c76381a1b6c07 100644 --- a/common/datablocks/src/kernels/data_block_slice.rs +++ b/common/datablocks/src/kernels/data_block_slice.rs @@ -2,7 +2,7 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::util::bit_util::ceil; +use common_datavalues::prelude::ceil; use common_exception::Result; use crate::DataBlock; diff --git a/common/datablocks/src/kernels/data_block_sort.rs b/common/datablocks/src/kernels/data_block_sort.rs index 1ccd5338a924e..f35030009eb15 100644 --- a/common/datablocks/src/kernels/data_block_sort.rs +++ b/common/datablocks/src/kernels/data_block_sort.rs @@ -2,7 +2,13 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::compute; +use std::iter::once; + +use common_arrow::arrow::array::growable::make_growable; +use common_arrow::arrow::array::Array; +use common_arrow::arrow::array::ArrayRef; +use common_arrow::arrow::compute::merge_sort::*; +use common_arrow::arrow::compute::sort as arrow_sort; use common_datavalues::prelude::*; use common_exception::ErrorCode; use common_exception::Result; @@ -23,10 +29,16 @@ impl DataBlock { ) -> Result { let order_columns = sort_columns_descriptions .iter() - .map(|f| { - Ok(compute::SortColumn { - values: block.try_array_by_name(&f.column_name)?.get_array_ref(), - options: Some(compute::SortOptions { + .map(|f| Ok(block.try_array_by_name(&f.column_name)?.get_array_ref())) + .collect::>>()?; + + let order_arrays = sort_columns_descriptions + .iter() + .zip(order_columns.iter()) + .map(|(f, array)| { + Ok(arrow_sort::SortColumn { + values: array.as_ref(), + options: Some(arrow_sort::SortOptions { descending: !f.asc, nulls_first: f.nulls_first, }), @@ -34,7 +46,7 @@ impl DataBlock { }) .collect::>>()?; - let indices = compute::lexsort_to_indices(&order_columns, limit)?; + let indices = arrow_sort::lexsort_to_indices(&order_arrays, limit)?; DataBlock::block_take_by_indices(block, &[], indices.values()) } @@ -52,45 +64,132 @@ impl DataBlock { return Ok(lhs.clone()); } - let mut sort_columns = vec![]; - for block in [lhs, rhs].iter() { - let columns = sort_columns_descriptions - .iter() - .map(|f| Ok(block.try_column_by_name(&f.column_name)?.clone())) - .collect::>>()?; - sort_columns.push(columns); - } - - let sort_options = sort_columns_descriptions + let sort_arrays = sort_columns_descriptions .iter() .map(|f| { - Ok(compute::SortOptions { - descending: !f.asc, - nulls_first: f.nulls_first, - }) + let left = lhs.try_column_by_name(&f.column_name)?.clone(); + let left = left.to_array()?; + + let right = rhs.try_column_by_name(&f.column_name)?.clone(); + let right = right.to_array()?; + + Ok(vec![left.get_array_ref(), right.get_array_ref()]) }) .collect::>>()?; - let indices = DataColumnCommon::merge_indices( - &sort_columns[0], - &sort_columns[1], - &sort_options, - limit, - )?; + let sort_dyn_arrays = sort_arrays + .iter() + .map(|f| vec![f[0].as_ref(), f[1].as_ref()]) + .collect::>(); + + let sort_options = sort_columns_descriptions + .iter() + .map(|f| arrow_sort::SortOptions { + descending: !f.asc, + nulls_first: f.nulls_first, + }) + .collect::>(); - let indices = match limit { - Some(limit) => &indices[0..limit.min(indices.len())], - _ => &indices, - }; + let sort_options_with_array = sort_dyn_arrays + .iter() + .zip(sort_options.iter()) + .map(|(s, opt)| { + let paris: (&[&dyn Array], &SortOptions) = (s, opt); + paris + }) + .collect::>(); + + let comparator = build_comparator(&sort_options_with_array)?; + let lhs_indices = (0, 0, lhs.num_rows()); + let rhs_indices = (1, 0, rhs.num_rows()); + let slices = merge_sort_slices(once(&lhs_indices), once(&rhs_indices), &comparator); + let slices = Self::materialize_merge_indices(slices, limit); - let arrays = lhs - .columns() + let fields = lhs.schema().fields(); + let columns = fields .iter() - .zip(rhs.columns().iter()) - .map(|(a, b)| DataColumnCommon::merge_columns(a, b, indices)) + .map(|f| { + let left = lhs.try_column_by_name(f.name())?; + let right = rhs.try_column_by_name(f.name())?; + + let left = left.to_array()?; + let right = right.to_array()?; + + let taked = Self::take_arrays_by_slices( + &[ + left.get_array_ref().as_ref(), + right.get_array_ref().as_ref(), + ], + &slices, + limit, + ); + let taked: ArrayRef = Arc::from(taked); + + Ok(DataColumn::Array(taked.into_series())) + }) .collect::>>()?; - Ok(DataBlock::create(lhs.schema().clone(), arrays)) + Ok(DataBlock::create(lhs.schema().clone(), columns)) + } + + fn materialize_merge_indices< + 'a, + L: Iterator, + R: Iterator, + >( + slices: MergeSortSlices<'a, L, R>, + limit: Option, + ) -> Vec { + match limit { + Some(limit) => { + let mut v = Vec::with_capacity(limit); + let mut current_len = 0; + for (index, start, len) in slices { + v.push((index, start, len)); + + if len + current_len >= limit { + break; + } else { + current_len += len; + } + } + + v + } + None => slices.into_iter().collect(), + } + } + + pub fn take_arrays_by_slices( + arrays: &[&dyn Array], + slices: &[MergeSlice], + limit: Option, + ) -> Box { + let slices = slices.iter(); + let len = arrays.iter().map(|array| array.len()).sum(); + + let limit = limit.unwrap_or(len); + let limit = limit.min(len); + let mut growable = make_growable(arrays, false, limit); + + if limit != len { + let mut current_len = 0; + for (index, start, len) in slices { + if len + current_len >= limit { + growable.extend(*index, *start, limit - current_len); + break; + } else { + growable.extend(*index, *start, *len); + current_len += len; + } + } + } else { + for (index, start, len) in slices { + growable.extend(*index, *start, *len); + } + } + + growable.as_box() } pub fn merge_sort_blocks( diff --git a/common/datablocks/src/kernels/data_block_sort_test.rs b/common/datablocks/src/kernels/data_block_sort_test.rs index 23a38145872a8..3bc4bb063c1f5 100644 --- a/common/datablocks/src/kernels/data_block_sort_test.rs +++ b/common/datablocks/src/kernels/data_block_sort_test.rs @@ -62,3 +62,48 @@ fn test_data_block_sort() -> Result<()> { } Ok(()) } + +#[test] +fn test_data_block_merge_sort() -> Result<()> { + let schema = DataSchemaRefExt::create(vec![ + DataField::new("a", DataType::Int64, false), + DataField::new("b", DataType::Utf8, false), + ]); + + let raw1 = DataBlock::create_by_array(schema.clone(), vec![ + Series::new(vec![3, 5, 7]), + Series::new(vec!["b1", "b2", "b3"]), + ]); + + let raw2 = DataBlock::create_by_array(schema.clone(), vec![ + Series::new(vec![2, 4, 6]), + Series::new(vec!["b4", "b5", "b6"]), + ]); + + { + let options = vec![SortColumnDescription { + column_name: "a".to_owned(), + asc: true, + nulls_first: false, + }]; + let results = DataBlock::merge_sort_block(&raw1, &raw2, &options, None)?; + + assert_eq!(raw1.schema(), results.schema()); + + let expected = vec![ + "+---+----+", + "| a | b |", + "+---+----+", + "| 2 | b4 |", + "| 3 | b1 |", + "| 4 | b5 |", + "| 5 | b2 |", + "| 6 | b6 |", + "| 7 | b3 |", + "+---+----+", + ]; + crate::assert_blocks_eq(expected, &[results]); + } + + Ok(()) +} diff --git a/common/datavalues/src/arrays/arithmetic.rs b/common/datavalues/src/arrays/arithmetic.rs index d273b3f05bb74..7e9c6523d16e3 100644 --- a/common/datavalues/src/arrays/arithmetic.rs +++ b/common/datavalues/src/arrays/arithmetic.rs @@ -14,66 +14,18 @@ use std::sync::Arc; use common_arrow::arrow::array::Array; use common_arrow::arrow::array::ArrayRef; use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::compute; -use common_arrow::arrow::compute::divide_scalar; +use common_arrow::arrow::compute::arithmetics::basic; +use common_arrow::arrow::compute::arithmetics::negate; use common_arrow::arrow::error::ArrowError; use common_exception::ErrorCode; use common_exception::Result; -use num::Num; -use num::NumCast; -use num::One; use num::ToPrimitive; -use num::Zero; use crate::arrays::ops::*; use crate::arrays::DataArray; use crate::prelude::*; use crate::*; -macro_rules! apply_operand_on_array_by_iter { - - ($self:ident, $rhs:ident, $operand:tt) => { - { - match ($self.null_count(), $rhs.null_count()) { - (0, 0) => { - let a: NoNull> = $self - .into_no_null_iter() - .zip($rhs.into_no_null_iter()) - .map(|(left, right)| left $operand right) - .collect(); - a.into_inner() - }, - (0, _) => { - $self - .into_no_null_iter() - .zip($rhs.downcast_iter()) - .map(|(left, opt_right)| opt_right.map(|right| left $operand right)) - .collect() - }, - (_, 0) => { - $self - .downcast_iter() - .zip($rhs.into_no_null_iter()) - .map(|(opt_left, right)| opt_left.map(|left| left $operand right)) - .collect() - }, - (_, _) => { - $self.downcast_iter() - .zip($rhs.downcast_iter()) - .map(|(opt_left, opt_right)| match (opt_left, opt_right) { - (None, None) => None, - (None, Some(_)) => None, - (Some(_), None) => None, - (Some(left), Some(right)) => Some(left $operand right), - }) - .collect() - - } - } - } - } -} - fn arithmetic_helper( lhs: &DataArray, rhs: &DataArray, @@ -88,9 +40,9 @@ where + Div + num::Zero, Kernel: Fn( - &PrimitiveArray, - &PrimitiveArray, - ) -> std::result::Result, ArrowError>, + &PrimitiveArray, + &PrimitiveArray, + ) -> std::result::Result, ArrowError>, F: Fn(T::Native, T::Native) -> T::Native, { let ca = match (lhs.len(), rhs.len()) { @@ -132,7 +84,7 @@ where type Output = Result>; fn add(self, rhs: Self) -> Self::Output { - arithmetic_helper(self, rhs, compute::add, |lhs, rhs| lhs + rhs) + arithmetic_helper(self, rhs, basic::add::add, |lhs, rhs| lhs + rhs) } } @@ -149,7 +101,7 @@ where type Output = Result>; fn sub(self, rhs: Self) -> Self::Output { - arithmetic_helper(self, rhs, compute::subtract, |lhs, rhs| lhs - rhs) + arithmetic_helper(self, rhs, basic::sub::sub, |lhs, rhs| lhs - rhs) } } @@ -166,7 +118,7 @@ where type Output = Result>; fn mul(self, rhs: Self) -> Self::Output { - arithmetic_helper(self, rhs, compute::multiply, |lhs, rhs| lhs * rhs) + arithmetic_helper(self, rhs, basic::mul::mul, |lhs, rhs| lhs * rhs) } } @@ -184,14 +136,7 @@ where type Output = Result>; fn div(self, rhs: Self) -> Self::Output { - if rhs.len() == 1 { - let result = Arc::new(compute::divide_scalar( - self.as_ref(), - rhs.as_ref().value(0), - )?) as ArrayRef; - return Ok(result.into()); - } - arithmetic_helper(self, rhs, compute::divide, |lhs, rhs| lhs / rhs) + arithmetic_helper(self, rhs, basic::div::div, |lhs, rhs| lhs / rhs) } } @@ -234,7 +179,7 @@ where } _ => { - let array = arithmetic_helper(self, rhs, compute::modulus, |lhs, rhs| lhs % rhs)?; + let array = arithmetic_helper(self, rhs, basic::rem::rem, |lhs, rhs| lhs % rhs)?; Ok(array.into_series()) } } @@ -258,26 +203,26 @@ where let arr = &*self.array; let result = unsafe { match self.data_type() { - DataType::Int8 => Ok(Arc::new(compute::negate( - &*(arr as *const dyn Array as *const PrimitiveArray), - )?) as ArrayRef), - - DataType::Int16 => Ok(Arc::new(compute::negate( - &*(arr as *const dyn Array as *const PrimitiveArray), - )?) as ArrayRef), - - DataType::Int32 => Ok(Arc::new(compute::negate( - &*(arr as *const dyn Array as *const PrimitiveArray), - )?) as ArrayRef), - DataType::Int64 => Ok(Arc::new(compute::negate( - &*(arr as *const dyn Array as *const PrimitiveArray), - )?) as ArrayRef), - DataType::Float32 => Ok(Arc::new(compute::negate( - &*(arr as *const dyn Array as *const PrimitiveArray), - )?) as ArrayRef), - DataType::Float64 => Ok(Arc::new(compute::negate( - &*(arr as *const dyn Array as *const PrimitiveArray), - )?) as ArrayRef), + DataType::Int8 => Ok(Arc::new(negate( + &*(arr as *const dyn Array as *const PrimitiveArray), + )) as ArrayRef), + + DataType::Int16 => Ok(Arc::new(negate( + &*(arr as *const dyn Array as *const PrimitiveArray), + )) as ArrayRef), + + DataType::Int32 => Ok(Arc::new(negate( + &*(arr as *const dyn Array as *const PrimitiveArray), + )) as ArrayRef), + DataType::Int64 => Ok(Arc::new(negate( + &*(arr as *const dyn Array as *const PrimitiveArray), + )) as ArrayRef), + DataType::Float32 => Ok(Arc::new(negate( + &*(arr as *const dyn Array as *const PrimitiveArray), + )) as ArrayRef), + DataType::Float64 => Ok(Arc::new(negate( + &*(arr as *const dyn Array as *const PrimitiveArray), + )) as ArrayRef), _ => Err(ErrorCode::IllegalDataType(format!( "DataType {:?} is Unsupported for neg op", @@ -290,93 +235,6 @@ where } } -// Operands on DataArray & Num - -impl Add for &DataArray -where - T: DFNumericType, - T::Native: NumCast, - N: Num + ToPrimitive, - T::Native: Add, -{ - type Output = Result>; - - fn add(self, rhs: N) -> Self::Output { - let adder: T::Native = NumCast::from(rhs).unwrap(); - Ok(self.apply(|val| val + adder)) - } -} - -impl Sub for &DataArray -where - T: DFNumericType, - T::Native: NumCast, - N: Num + ToPrimitive, - T::Native: Sub, -{ - type Output = Result>; - - fn sub(self, rhs: N) -> Self::Output { - let subber: T::Native = NumCast::from(rhs).unwrap(); - Ok(self.apply(|val| val - subber)) - } -} - -impl Div for &DataArray -where - T: DFNumericType, - T::Native: NumCast - + Div - + One - + Zero - + Rem - + Sub, - N: Num + ToPrimitive, -{ - type Output = Result>; - - fn div(self, rhs: N) -> Self::Output { - let rhs: T::Native = NumCast::from(rhs).expect("could not cast"); - Ok(self.apply_kernel(|arr| Arc::new(divide_scalar(arr, rhs).unwrap()))) - } -} - -impl Mul for &DataArray -where - T: DFNumericType, - T::Native: NumCast, - N: Num + ToPrimitive, - T::Native: Mul, -{ - type Output = Result>; - - fn mul(self, rhs: N) -> Self::Output { - let multiplier: T::Native = NumCast::from(rhs).unwrap(); - Ok(self.apply(|val| val * multiplier)) - } -} - -impl Rem for &DataArray -where - T: DFNumericType, - T::Native: NumCast, - N: Num + ToPrimitive, - T::Native: Add - + Sub - + Mul - + Div - + Rem - + Zero - + One, -{ - type Output = Result>; - - fn rem(self, rhs: N) -> Self::Output { - let rhs: T::Native = NumCast::from(rhs).expect("could not cast"); - Ok(self.apply_kernel(|arr| Arc::new(compute::modulus_scalar(arr, rhs).unwrap()))) - } -} - fn concat_strings(l: &str, r: &str) -> String { // fastest way to concat strings according to https://github.com/hoodie/concatenation_benchmarks-rs let mut s = String::with_capacity(l.len() + r.len()); @@ -400,8 +258,8 @@ impl Add for &DFUtf8Array { // todo! add no_null variants. Need 4 paths. Ok(self - .into_iter() - .zip(rhs.into_iter()) + .downcast_iter() + .zip(rhs.downcast_iter()) .map(|(opt_l, opt_r)| match (opt_l, opt_r) { (Some(l), Some(r)) => Some(concat_strings(l, r)), _ => None, @@ -428,7 +286,7 @@ impl Add<&str> for &DFUtf8Array { .map(|l| concat_strings(l, rhs)) .collect(), _ => self - .into_iter() + .downcast_iter() .map(|opt_l| opt_l.map(|l| concat_strings(l, rhs))) .collect(), }) @@ -452,13 +310,13 @@ where fn pow_f32(&self, exp: f32) -> DFFloat32Array { self.cast::() .expect("f32 array") - .apply_kernel(|arr| Arc::new(compute::powf_scalar(arr, exp).unwrap())) + .apply_kernel(|arr| Arc::new(basic::pow::powf_scalar(arr, exp))) } fn pow_f64(&self, exp: f64) -> DFFloat64Array { self.cast::() .expect("f64 array") - .apply_kernel(|arr| Arc::new(compute::powf_scalar(arr, exp).unwrap())) + .apply_kernel(|arr| Arc::new(basic::pow::powf_scalar(arr, exp))) } } diff --git a/common/datavalues/src/arrays/arrow_array.rs b/common/datavalues/src/arrays/arrow_array.rs deleted file mode 100644 index 4f3f768331fb3..0000000000000 --- a/common/datavalues/src/arrays/arrow_array.rs +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright 2020-2021 The Datafuse Authors. -// -// SPDX-License-Identifier: Apache-2.0. - -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::ArrayData; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::datatypes::ArrowPrimitiveType; -use common_arrow::arrow::datatypes::DataType; -use num::Num; - -/// Cheaply get the null mask as BooleanArray. -pub trait IsNull { - fn is_null_mask(&self) -> BooleanArray; - fn is_not_null_mask(&self) -> BooleanArray; -} - -impl IsNull for &dyn Array { - fn is_null_mask(&self) -> BooleanArray { - if self.null_count() == 0 { - (0..self.len()).map(|_| Some(false)).collect() - } else { - let data = self.data(); - let valid = data.null_buffer().unwrap(); - let invert = !valid; - - let array_data = ArrayData::builder(DataType::Boolean) - .len(self.len()) - .offset(self.offset()) - .add_buffer(invert) - .build(); - BooleanArray::from(array_data) - } - } - - fn is_not_null_mask(&self) -> BooleanArray { - if self.null_count() == 0 { - (0..self.len()).map(|_| Some(true)).collect() - } else { - let data = self.data(); - let valid = data.null_buffer().unwrap().clone(); - - let array_data = ArrayData::builder(DataType::Boolean) - .len(self.len()) - .offset(self.offset()) - .add_buffer(valid) - .build(); - BooleanArray::from(array_data) - } - } -} - -pub trait GetValues { - fn get_values(&self) -> &[T::Native] - where - T: ArrowPrimitiveType, - T::Native: Num; -} - -impl GetValues for ArrayData { - fn get_values(&self) -> &[T::Native] - where - T: ArrowPrimitiveType, - T::Native: Num, - { - debug_assert_eq!(&T::DATA_TYPE, self.data_type()); - // the first buffer is the value array - let value_buf = &self.buffers()[0]; - let offset = self.offset(); - let vals = unsafe { value_buf.typed_data::() }; - &vals[offset..offset + self.len()] - } -} - -impl GetValues for &dyn Array { - fn get_values(&self) -> &[T::Native] - where - T: ArrowPrimitiveType, - T::Native: Num, - { - self.data_ref().get_values::() - } -} - -impl GetValues for ArrayRef { - fn get_values(&self) -> &[T::Native] - where - T: ArrowPrimitiveType, - T::Native: Num, - { - self.data_ref().get_values::() - } -} - -pub trait ToPrimitive { - fn into_primitive_array(self) -> PrimitiveArray - where T: ArrowPrimitiveType; -} - -impl ToPrimitive for ArrayData { - fn into_primitive_array(self) -> PrimitiveArray - where T: ArrowPrimitiveType { - PrimitiveArray::from(self) - } -} - -impl ToPrimitive for &dyn Array { - fn into_primitive_array(self) -> PrimitiveArray - where T: ArrowPrimitiveType { - self.data().clone().into_primitive_array() - } -} diff --git a/common/datavalues/src/arrays/builders/arrow_builder.rs b/common/datavalues/src/arrays/builders/arrow_builder.rs deleted file mode 100644 index 71e5cb6dc50d8..0000000000000 --- a/common/datavalues/src/arrays/builders/arrow_builder.rs +++ /dev/null @@ -1,404 +0,0 @@ -// Copyright 2020-2021 The Datafuse Authors. -// -// SPDX-License-Identifier: Apache-2.0. - -use std::any::Any; -use std::mem; -use std::sync::Arc; - -use common_arrow::arrow::array::ArrayBuilder; -use common_arrow::arrow::array::ArrayData; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; -use common_arrow::arrow::buffer::Buffer; -use common_arrow::arrow::buffer::MutableBuffer; -use common_arrow::arrow::datatypes::ArrowPrimitiveType; -use common_arrow::arrow::datatypes::DataType; - -use crate::bit_util; -use crate::vec::AlignedVec; - -#[derive(Debug)] -pub struct BooleanBufferBuilder { - buffer: MutableBuffer, - len: usize, -} - -impl BooleanBufferBuilder { - #[inline] - pub fn new(capacity: usize) -> Self { - let byte_capacity = bit_util::ceil(capacity, 8); - let buffer = MutableBuffer::from_len_zeroed(byte_capacity); - Self { buffer, len: 0 } - } - - pub fn len(&self) -> usize { - self.len - } - - pub fn is_empty(&self) -> bool { - self.len == 0 - } - - pub fn capacity(&self) -> usize { - self.buffer.capacity() * 8 - } - - #[inline] - pub fn advance(&mut self, additional: usize) { - let new_len = self.len + additional; - let new_len_bytes = bit_util::ceil(new_len, 8); - if new_len_bytes > self.buffer.len() { - self.buffer.resize(new_len_bytes, 0); - } - self.len = new_len; - } - - /// Reserve space to at least `additional` new bits. - /// Capacity will be `>= self.len() + additional`. - /// New bytes are uninitialized and reading them is undefined behavior. - #[inline] - pub fn reserve(&mut self, additional: usize) { - let capacity = self.len + additional; - if capacity > self.capacity() { - // convert differential to bytes - let additional = bit_util::ceil(capacity, 8) - self.buffer.len(); - self.buffer.reserve(additional); - } - } - - #[inline] - pub fn append(&mut self, v: bool) { - self.advance(1); - if v { - unsafe { bit_util::set_bit_raw(self.buffer.as_mut_ptr(), self.len - 1) }; - } - } - - #[inline] - pub fn append_n(&mut self, additional: usize, v: bool) { - self.advance(additional); - if additional > 0 && v { - let offset = self.len() - additional; - (0..additional).for_each(|i| unsafe { - bit_util::set_bit_raw(self.buffer.as_mut_ptr(), offset + i) - }) - } - } - - #[inline] - pub fn append_slice(&mut self, slice: &[bool]) { - let additional = slice.len(); - self.advance(additional); - - let offset = self.len() - additional; - for (i, v) in slice.iter().enumerate() { - if *v { - unsafe { bit_util::set_bit_raw(self.buffer.as_mut_ptr(), offset + i) } - } - } - } - - pub fn shrink_to_fit(&mut self) { - let byte_len = bit_util::ceil(self.len(), 8); - self.buffer.resize(byte_len, 0) - } - - #[inline] - pub fn finish(&mut self) -> Buffer { - let buf = std::mem::replace(&mut self.buffer, MutableBuffer::new(0)); - self.len = 0; - buf.into() - } -} - -/// Array builder for fixed-width primitive types -#[derive(Debug)] -pub struct ArrowBooleanArrayBuilder { - values_builder: BooleanBufferBuilder, - bitmap_builder: BooleanBufferBuilder, -} - -impl ArrowBooleanArrayBuilder { - /// Creates a new primitive array builder - pub fn new(capacity: usize) -> Self { - Self { - values_builder: BooleanBufferBuilder::new(capacity), - bitmap_builder: BooleanBufferBuilder::new(capacity), - } - } - - pub fn new_no_nulls(capacity: usize) -> Self { - Self { - values_builder: BooleanBufferBuilder::new(capacity), - bitmap_builder: BooleanBufferBuilder::new(0), - } - } - - /// Returns the capacity of this builder measured in slots of type `T` - pub fn capacity(&self) -> usize { - self.values_builder.capacity() - } - - /// Appends a value of type `T` into the builder - pub fn append_value(&mut self, v: bool) { - self.bitmap_builder.append(true); - self.values_builder.append(v); - } - - /// Appends a null slot into the builder - pub fn append_null(&mut self) { - self.bitmap_builder.append(false); - self.values_builder.advance(1); - } - - /// Appends an `Option` into the builder - pub fn append_option(&mut self, v: Option) { - match v { - None => self.append_null(), - Some(v) => self.append_value(v), - }; - } - - /// Appends a slice of type `T` into the builder - pub fn append_slice(&mut self, v: &[bool]) { - self.bitmap_builder.append_n(v.len(), true); - self.values_builder.append_slice(v); - } - - /// Appends values from a slice of type `T` and a validity boolean slice - pub fn append_values(&mut self, values: &[bool], is_valid: &[bool]) { - assert_eq!(values.len(), is_valid.len()); - self.bitmap_builder.append_slice(is_valid); - self.values_builder.append_slice(values); - } - - pub fn shrink_to_fit(&mut self) { - self.values_builder.shrink_to_fit(); - self.bitmap_builder.shrink_to_fit(); - } - - pub fn finish_with_null_buffer(&mut self, buffer: Buffer) -> BooleanArray { - self.shrink_to_fit(); - let len = self.len(); - let data = ArrayData::builder(DataType::Boolean) - .len(len) - .add_buffer(self.values_builder.finish()) - .null_bit_buffer(buffer) - .build(); - BooleanArray::from(data) - } - - /// Builds the [BooleanArray] and reset this builder. - pub fn finish(&mut self) -> BooleanArray { - self.shrink_to_fit(); - let len = self.len(); - let null_bit_buffer = self.bitmap_builder.finish(); - let null_count = len - null_bit_buffer.count_set_bits(); - let mut builder = ArrayData::builder(DataType::Boolean) - .len(len) - .add_buffer(self.values_builder.finish()); - if null_count > 0 { - builder = builder.null_bit_buffer(null_bit_buffer); - } - let data = builder.build(); - BooleanArray::from(data) - } -} - -impl ArrayBuilder for ArrowBooleanArrayBuilder { - /// Returns the builder as a non-mutable `Any` reference. - fn as_any(&self) -> &dyn Any { - self - } - - /// Returns the builder as a mutable `Any` reference. - fn as_any_mut(&mut self) -> &mut dyn Any { - self - } - - /// Returns the boxed builder as a box of `Any`. - fn into_box_any(self: Box) -> Box { - self - } - - /// Returns the number of array slots in the builder - fn len(&self) -> usize { - self.values_builder.len() - } - - /// Returns whether the number of array slots is zero - fn is_empty(&self) -> bool { - self.values_builder.is_empty() - } - - /// Builds the array and reset this builder. - fn finish(&mut self) -> ArrayRef { - Arc::new(self.finish()) - } -} - -/// An arrow primitive builder that is faster than Arrow's native builder because it uses Rust Vec's -/// as buffer -pub struct ArrowPrimitiveArrayBuilder -where - T: ArrowPrimitiveType, - T::Native: Default, -{ - values: AlignedVec, - bitmap_builder: BooleanBufferBuilder, - null_count: usize, -} - -impl ArrowPrimitiveArrayBuilder -where - T: ArrowPrimitiveType, - T::Native: Default, -{ - pub fn new(capacity: usize) -> Self { - let values = AlignedVec::::with_capacity_aligned(capacity); - let bitmap_builder = BooleanBufferBuilder::new(capacity); - - Self { - values, - bitmap_builder, - null_count: 0, - } - } - - pub fn new_no_nulls(capacity: usize) -> Self { - let values = AlignedVec::::with_capacity_aligned(capacity); - let bitmap_builder = BooleanBufferBuilder::new(0); - - Self { - values, - bitmap_builder, - null_count: 0, - } - } - - /// Appends a value of type `T::Native` into the builder - #[inline] - pub fn append_value(&mut self, v: T::Native) { - self.values.push(v); - self.bitmap_builder.append(true); - } - - #[inline] - pub fn append_slice(&mut self, other: &[T::Native]) { - self.values.extend_from_slice(other) - } - - /// Appends a null slot into the builder - #[inline] - pub fn append_null(&mut self) { - self.bitmap_builder.append(false); - self.values.push(Default::default()); - self.null_count += 1; - } - - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - self.bitmap_builder.shrink_to_fit(); - } - - pub fn finish_with_null_buffer(&mut self, buffer: Buffer) -> PrimitiveArray { - self.shrink_to_fit(); - let values = mem::take(&mut self.values); - values.into_primitive_array(Some(buffer)) - } - - /// Build the array and reset this Builder - pub fn finish(&mut self) -> PrimitiveArray { - self.shrink_to_fit(); - let values = mem::take(&mut self.values); - let null_bit_buffer = self.bitmap_builder.finish(); - let buf = if self.null_count == 0 { - None - } else { - Some(null_bit_buffer) - }; - values.into_primitive_array(buf) - } -} - -impl ArrayBuilder for ArrowPrimitiveArrayBuilder -where T: ArrowPrimitiveType -{ - fn len(&self) -> usize { - self.values.len() - } - - fn is_empty(&self) -> bool { - self.values.is_empty() - } - - fn finish(&mut self) -> ArrayRef { - Arc::new(ArrowPrimitiveArrayBuilder::finish(self)) - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn as_any_mut(&mut self) -> &mut dyn Any { - self - } - - fn into_box_any(self: Box) -> Box { - self - } -} - -#[derive(Debug)] -pub struct ArrowNoNullStringBuilder { - values: AlignedVec, - offsets: AlignedVec, -} - -impl ArrowNoNullStringBuilder { - pub fn with_capacity(values_capacity: usize, list_capacity: usize) -> Self { - let mut offsets = AlignedVec::with_capacity_aligned(list_capacity + 1); - offsets.push(0); - Self { - values: AlignedVec::with_capacity_aligned(values_capacity), - offsets, - } - } - - /// Extends with values and offsets. - pub fn extend_from_slices(&mut self, values: &[u8], offsets: &[i32]) { - self.values.extend_from_slice(values); - self.offsets.extend_from_slice(offsets); - } - - #[inline] - pub fn append_value(&mut self, value: &str) { - self.values.extend_from_slice(value.as_bytes()); - self.offsets.push(self.values.len() as i32); - } - - /// Builds the `StringArray` and reset this builder. - pub fn finish(&mut self) -> StringArray { - // values are u8 typed - let values = mem::take(&mut self.values); - // offsets are i64 typed - let offsets = mem::take(&mut self.offsets); - let offsets_len = offsets.len() - 1; - // buffers are u8 typed - let buf_offsets = offsets.into_arrow_buffer(); - let buf_values = values.into_arrow_buffer(); - assert_eq!(buf_values.len(), buf_values.capacity()); - assert_eq!(buf_offsets.len(), buf_offsets.capacity()); - - // note that the arrays are already shrunk when transformed to an arrow buffer. - let arraydata = ArrayData::builder(DataType::Utf8) - .len(offsets_len) - .add_buffer(buf_offsets) - .add_buffer(buf_values) - .build(); - StringArray::from(arraydata) - } -} diff --git a/common/datavalues/src/arrays/builders/builder.rs b/common/datavalues/src/arrays/builders/builder.rs index 0d649b750beb3..3466bef477c67 100644 --- a/common/datavalues/src/arrays/builders/builder.rs +++ b/common/datavalues/src/arrays/builders/builder.rs @@ -2,22 +2,10 @@ // // SPDX-License-Identifier: Apache-2.0. -use std::sync::Arc; - -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BinaryBuilder; -use common_arrow::arrow::array::BooleanBuilder; -use common_arrow::arrow::array::ListBuilder; -use common_arrow::arrow::array::PrimitiveBuilder; -use common_arrow::arrow::array::StringBuilder; -use common_arrow::arrow::buffer::Buffer; +use common_arrow::arrow::array::*; use num::Num; -use super::ArrowBooleanArrayBuilder; -use super::ArrowPrimitiveArrayBuilder; use crate::arrays::DataArray; -use crate::arrays::GetValues; use crate::data_df_type::*; use crate::prelude::*; use crate::utils::get_iter_capacity; @@ -43,32 +31,32 @@ pub trait ArrayBuilder { } pub struct BooleanArrayBuilder { - builder: BooleanBuilder, + builder: MutableBooleanArray, } impl ArrayBuilder for BooleanArrayBuilder { /// Appends a value of type `T` into the builder #[inline] fn append_value(&mut self, v: bool) { - self.builder.append_value(v).unwrap(); + self.builder.push(Some(v)) } /// Appends a null slot into the builder #[inline] fn append_null(&mut self) { - self.builder.append_null().unwrap(); + self.builder.push_null(); } fn finish(&mut self) -> DFBooleanArray { - let array = Arc::new(self.builder.finish()) as ArrayRef; + let array = self.builder.as_arc(); array.into() } } impl BooleanArrayBuilder { - pub fn new(capacity: usize) -> Self { + pub fn with_capacity(capacity: usize) -> Self { BooleanArrayBuilder { - builder: BooleanBuilder::new(capacity), + builder: MutableBooleanArray::with_capacity(capacity), } } } @@ -78,7 +66,7 @@ where T: DFNumericType, T::Native: Default, { - builder: PrimitiveBuilder, + builder: MutablePrimitiveArray, } impl ArrayBuilder for PrimitiveArrayBuilder @@ -89,17 +77,17 @@ where /// Appends a value of type `T` into the builder #[inline] fn append_value(&mut self, v: T::Native) { - self.builder.append_value(v).unwrap(); + self.builder.push(Some(v)) } /// Appends a null slot into the builder #[inline] fn append_null(&mut self) { - self.builder.append_null().unwrap(); + self.builder.push_null(); } fn finish(&mut self) -> DataArray { - let array = Arc::new(self.builder.finish()) as ArrayRef; + let array = self.builder.as_arc(); array.into() } @@ -108,9 +96,9 @@ where impl PrimitiveArrayBuilder where T: DFNumericType { - pub fn new(capacity: usize) -> Self { + pub fn with_capacity(capacity: usize) -> Self { PrimitiveArrayBuilder { - builder: PrimitiveBuilder::::new(capacity), + builder: MutablePrimitiveArray::::with_capacity(capacity), } } } @@ -125,8 +113,7 @@ pub type DFUInt64ArrayBuilder = PrimitiveArrayBuilder; pub type DFInt64ArrayBuilder = PrimitiveArrayBuilder; pub struct Utf8ArrayBuilder { - pub builder: StringBuilder, - pub capacity: usize, + pub builder: MutableUtf8Array, } impl Utf8ArrayBuilder { @@ -135,24 +122,22 @@ impl Utf8ArrayBuilder { /// # Arguments /// /// * `capacity` - Number of string elements in the final array. - /// * `bytes_capacity` - Number of bytes needed to store the string values. - pub fn new(capacity: usize, bytes_capacity: usize) -> Self { + pub fn with_capacity(bytes_capacity: usize) -> Self { Utf8ArrayBuilder { - builder: StringBuilder::with_capacity(bytes_capacity, capacity), - capacity, + builder: MutableUtf8Array::with_capacity(bytes_capacity), } } /// Appends a value of type `T` into the builder #[inline] pub fn append_value>(&mut self, v: S) { - self.builder.append_value(v.as_ref()).unwrap(); + self.builder.push(Some(v)) } /// Appends a null slot into the builder #[inline] pub fn append_null(&mut self) { - self.builder.append_null().unwrap(); + self.builder.push_null(); } #[inline] @@ -164,23 +149,11 @@ impl Utf8ArrayBuilder { } pub fn finish(&mut self) -> DFUtf8Array { - let array = Arc::new(self.builder.finish()) as ArrayRef; + let array = self.builder.as_arc(); array.into() } } -/// Get the null count and the null bitmap of the arrow array -pub fn get_bitmap(arr: &T) -> (usize, Option) { - let data = arr.data(); - ( - data.null_count(), - data.null_bitmap().as_ref().map(|bitmap| { - let buff = bitmap.buffer_ref(); - buff.clone() - }), - ) -} - pub trait NewDataArray { fn new_from_slice(v: &[N]) -> Self; fn new_from_opt_slice(opt_v: &[Option]) -> Self; @@ -204,7 +177,7 @@ where T: DFNumericType } fn new_from_opt_iter(it: impl Iterator>) -> DataArray { - let mut builder = PrimitiveArrayBuilder::new(get_iter_capacity(&it)); + let mut builder = PrimitiveArrayBuilder::with_capacity(get_iter_capacity(&it)); it.for_each(|opt| builder.append_option(opt)); builder.finish() } @@ -226,7 +199,7 @@ impl NewDataArray for DFBooleanArray { } fn new_from_opt_iter(it: impl Iterator>) -> DFBooleanArray { - let mut builder = BooleanArrayBuilder::new(get_iter_capacity(&it)); + let mut builder = BooleanArrayBuilder::with_capacity(get_iter_capacity(&it)); it.for_each(|opt| builder.append_option(opt)); builder.finish() } @@ -242,14 +215,12 @@ where S: AsRef { fn new_from_slice(v: &[S]) -> Self { let values_size = v.iter().fold(0, |acc, s| acc + s.as_ref().len()); - - let mut builder = StringBuilder::with_capacity(values_size, v.len()); + let mut builder = Utf8ArrayBuilder::with_capacity(values_size); v.iter().for_each(|val| { - builder.append_value(val.as_ref()).unwrap(); + builder.append_value(val.as_ref()); }); - let array = Arc::new(builder.finish()) as ArrayRef; - array.into() + builder.finish() } fn new_from_opt_slice(opt_v: &[Option]) -> Self { @@ -257,8 +228,7 @@ where S: AsRef Some(s) => acc + s.as_ref().len(), None => acc, }); - let mut builder = Utf8ArrayBuilder::new(values_size, opt_v.len()); - + let mut builder = Utf8ArrayBuilder::with_capacity(values_size); opt_v.iter().for_each(|opt| match opt { Some(v) => builder.append_value(v.as_ref()), None => builder.append_null(), @@ -268,7 +238,7 @@ where S: AsRef fn new_from_opt_iter(it: impl Iterator>) -> Self { let cap = get_iter_capacity(&it); - let mut builder = Utf8ArrayBuilder::new(cap, cap * 5); + let mut builder = Utf8ArrayBuilder::with_capacity(cap * 5); it.for_each(|opt| builder.append_option(opt)); builder.finish() } @@ -276,7 +246,7 @@ where S: AsRef /// Create a new DataArray from an iterator. fn new_from_iter(it: impl Iterator) -> Self { let cap = get_iter_capacity(&it); - let mut builder = Utf8ArrayBuilder::new(cap, cap * 5); + let mut builder = Utf8ArrayBuilder::with_capacity(cap * 5); it.for_each(|v| builder.append_value(v)); builder.finish() } @@ -289,15 +259,19 @@ pub trait ListBuilderTrait { fn finish(&mut self) -> DFListArray; } +type LargePrimitiveBuilder = MutableListArray>; +type LargeListUtf8Builder = MutableListArray>; +type LargeListBooleanBuilder = MutableListArray; + pub struct ListPrimitiveArrayBuilder where T: DFPrimitiveType { - pub builder: ListBuilder>, + pub builder: LargePrimitiveBuilder, } macro_rules! finish_list_builder { ($self:ident) => {{ - let arr = Arc::new($self.builder.finish()); + let arr = $self.builder.as_arc(); DFListArray::from(arr as ArrayRef) }}; } @@ -305,26 +279,27 @@ macro_rules! finish_list_builder { impl ListPrimitiveArrayBuilder where T: DFPrimitiveType { - pub fn new(values_builder: ArrowPrimitiveArrayBuilder, capacity: usize) -> Self { - let builder = ListBuilder::with_capacity(values_builder, capacity); + pub fn with_capacity(values_capacity: usize, capacity: usize) -> Self { + let values = MutablePrimitiveArray::::with_capacity(values_capacity); + let builder = LargePrimitiveBuilder::::new_with_capacity(values, capacity); + ListPrimitiveArrayBuilder { builder } } pub fn append_slice(&mut self, opt_v: Option<&[T::Native]>) { match opt_v { - Some(v) => { - self.builder.values().append_slice(v); - self.builder.append(true).expect("should not fail"); + Some(items) => { + let values = self.builder.mut_values(); + // Safety: + // A slice is a trusted length iterator + unsafe { values.extend_trusted_len_unchecked(items.iter().map(Some)) } + self.builder.try_push_valid().unwrap(); } None => { - self.builder.append(false).expect("should not fail"); + self.builder.push_null(); } } } - - pub fn append_null(&mut self) { - self.builder.append(false).expect("should not fail"); - } } impl ListBuilderTrait for ListPrimitiveArrayBuilder @@ -337,38 +312,27 @@ where match opt_s { Some(s) => self.append_series(s), None => { - self.builder.append(false).unwrap(); + self.builder.push_null(); } } } #[inline] fn append_null(&mut self) { - let builder = self.builder.values(); - builder.append_null(); - self.builder.append(true).unwrap(); + self.builder.push_null(); } #[inline] fn append_series(&mut self, s: &Series) { - let builder = self.builder.values(); let array = s.get_array_ref(); - let values = array.get_values::(); - // we would like to check if array has no null values. - // however at the time of writing there is a bug in append_slice, because it does not update - // the null bitmap - if s.null_count() == 0 { - builder.append_slice(values); - } else { - values.iter().enumerate().for_each(|(idx, v)| { - if array.is_valid(idx) { - builder.append_value(*v); - } else { - builder.append_null(); - } - }); - } - self.builder.append(true).unwrap(); + let arr = array + .as_any() + .downcast_ref::>() + .unwrap(); + + let values = self.builder.mut_values(); + unsafe { values.extend_trusted_len_unchecked(arr.into_iter()) } + self.builder.try_push_valid().unwrap(); } fn finish(&mut self) -> DFListArray { @@ -377,12 +341,15 @@ where } pub struct ListUtf8ArrayBuilder { - builder: ListBuilder, + builder: LargeListUtf8Builder, } +type LargeMutableUtf8Array = MutableUtf8Array; impl ListUtf8ArrayBuilder { - pub fn new(values_builder: StringBuilder, capacity: usize) -> Self { - let builder = ListBuilder::with_capacity(values_builder, capacity); + pub fn with_capacity(values_capacity: usize, capacity: usize) -> Self { + let values = LargeMutableUtf8Array::with_capacity(values_capacity); + let builder = LargeListUtf8Builder::new_with_capacity(values, capacity); + ListUtf8ArrayBuilder { builder } } } @@ -392,29 +359,22 @@ impl ListBuilderTrait for ListUtf8ArrayBuilder { match opt_s { Some(s) => self.append_series(s), None => { - self.builder.append(false).unwrap(); + self.builder.push_null(); } } } #[inline] fn append_null(&mut self) { - let builder = self.builder.values(); - builder.append_null().unwrap(); - self.builder.append(true).unwrap(); + self.builder.push_null(); } #[inline] fn append_series(&mut self, s: &Series) { let ca = s.utf8().unwrap(); - let value_builder = self.builder.values(); - for s in ca { - match s { - Some(s) => value_builder.append_value(s).unwrap(), - None => value_builder.append_null().unwrap(), - }; - } - self.builder.append(true).unwrap(); + let value_builder = self.builder.mut_values(); + value_builder.try_extend(ca).unwrap(); + self.builder.try_push_valid().unwrap(); } fn finish(&mut self) -> DFListArray { @@ -423,13 +383,13 @@ impl ListBuilderTrait for ListUtf8ArrayBuilder { } pub struct ListBooleanArrayBuilder { - builder: ListBuilder, + builder: LargeListBooleanBuilder, } impl ListBooleanArrayBuilder { - pub fn new(values_builder: ArrowBooleanArrayBuilder, capacity: usize) -> Self { - let builder = ListBuilder::with_capacity(values_builder, capacity); - + pub fn with_capacity(values_capacity: usize, capacity: usize) -> Self { + let values = MutableBooleanArray::with_capacity(values_capacity); + let builder = LargeListBooleanBuilder::new_with_capacity(values, capacity); Self { builder } } } @@ -439,29 +399,22 @@ impl ListBuilderTrait for ListBooleanArrayBuilder { match opt_s { Some(s) => self.append_series(s), None => { - self.builder.append(false).unwrap(); + self.builder.push_null(); } } } #[inline] fn append_null(&mut self) { - let builder = self.builder.values(); - builder.append_null(); - self.builder.append(true).unwrap(); + self.builder.push_null(); } #[inline] fn append_series(&mut self, s: &Series) { let ca = s.bool().unwrap(); - let value_builder = self.builder.values(); - for s in ca { - match s { - Some(s) => value_builder.append_value(s), - None => value_builder.append_null(), - }; - } - self.builder.append(true).unwrap(); + let value_builder = self.builder.mut_values(); + value_builder.try_extend(ca).unwrap(); + self.builder.try_push_valid().unwrap(); } fn finish(&mut self) -> DFListArray { @@ -476,22 +429,20 @@ pub fn get_list_builder( ) -> Box { macro_rules! get_primitive_builder { ($type:ty) => {{ - let values_builder = ArrowPrimitiveArrayBuilder::<$type>::new(value_capacity); - let builder = ListPrimitiveArrayBuilder::new(values_builder, list_capacity); + let builder = + ListPrimitiveArrayBuilder::<$type>::with_capacity(value_capacity, list_capacity); Box::new(builder) }}; } macro_rules! get_bool_builder { () => {{ - let values_builder = ArrowBooleanArrayBuilder::new(value_capacity); - let builder = ListBooleanArrayBuilder::new(values_builder, list_capacity); + let builder = ListBooleanArrayBuilder::with_capacity(value_capacity, list_capacity); Box::new(builder) }}; } macro_rules! get_utf8_builder { () => {{ - let values_builder = StringBuilder::with_capacity(value_capacity * 5, value_capacity); - let builder = ListUtf8ArrayBuilder::new(values_builder, list_capacity); + let builder = ListUtf8ArrayBuilder::with_capacity(value_capacity, list_capacity); Box::new(builder) }}; } @@ -504,26 +455,27 @@ pub fn get_list_builder( } pub struct BinaryArrayBuilder { - builder: BinaryBuilder, + builder: MutableBinaryArray, } impl BinaryArrayBuilder { - pub fn new(capacity: usize) -> Self { + pub fn with_capacity(capacity: usize) -> Self { Self { - builder: BinaryBuilder::new(capacity), + builder: MutableBinaryArray::::with_capacity(capacity), } } pub fn append_value(&mut self, value: impl AsRef<[u8]>) { - self.builder.append_value(value).unwrap(); + self.builder.push(Some(value)) } + #[inline] pub fn append_null(&mut self) { - self.builder.append_null().unwrap(); + self.builder.push_null(); } pub fn finish(&mut self) -> DataArray { - let array = self.builder.finish(); - DFBinaryArray::from_arrow_array(array) + let array = self.builder.as_arc(); + DFBinaryArray::from(array) } } diff --git a/common/datavalues/src/arrays/builders/mod.rs b/common/datavalues/src/arrays/builders/mod.rs index 26a4b7e475db5..ccb3b2f25970a 100644 --- a/common/datavalues/src/arrays/builders/mod.rs +++ b/common/datavalues/src/arrays/builders/mod.rs @@ -2,8 +2,6 @@ // // SPDX-License-Identifier: Apache-2.0. -mod arrow_builder; mod builder; -pub use arrow_builder::*; pub use builder::*; diff --git a/common/datavalues/src/arrays/comparison.rs b/common/datavalues/src/arrays/comparison.rs index c8263555ad323..17485703a355c 100644 --- a/common/datavalues/src/arrays/comparison.rs +++ b/common/datavalues/src/arrays/comparison.rs @@ -6,32 +6,36 @@ use std::fmt::Debug; use std::sync::Arc; use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; -use common_arrow::arrow::compute::kernels::comparison; -use common_arrow::arrow::compute::*; +use common_arrow::arrow::compute::comparison::boolean_compare_scalar; +use common_arrow::arrow::compute::comparison::compare; +use common_arrow::arrow::compute::comparison::primitive_compare_scalar; +use common_arrow::arrow::compute::comparison::utf8_compare_scalar; +use common_arrow::arrow::compute::comparison::Operator; +use common_arrow::arrow::compute::like; use common_exception::ErrorCode; use common_exception::Result; use num::Num; use num::NumCast; -use num::ToPrimitive; use super::DataArray; use crate::arrays::*; use crate::series::Series; -use crate::utils::NoNull; use crate::*; -pub trait ArrayCompare: Debug { - /// Check for equality and regard missing values as equal. - fn eq_missing(&self, _rhs: Rhs) -> Result { - Err(ErrorCode::BadDataValueType(format!( - "Unsupported compare operation: eq_missing for {:?}", - self, - ))) - } +pub trait NumComp: Num + NumCast + PartialOrd {} + +impl NumComp for f32 {} +impl NumComp for f64 {} +impl NumComp for i8 {} +impl NumComp for i16 {} +impl NumComp for i32 {} +impl NumComp for i64 {} +impl NumComp for u8 {} +impl NumComp for u16 {} +impl NumComp for u32 {} +impl NumComp for u64 {} +pub trait ArrayCompare: Debug { /// Check for equality. fn eq(&self, _rhs: Rhs) -> Result { Err(ErrorCode::BadDataValueType(format!( @@ -96,70 +100,38 @@ pub trait ArrayCompare: Debug { } impl DataArray -where T: DFNumericType +where + T: DFNumericType, + T::Native: NumComp, { /// First ensure that the Arrays of lhs and rhs match and then iterates over the Arrays and applies /// the comparison operator. - fn comparison( - &self, - rhs: &DataArray, - operator: impl Fn( - &PrimitiveArray, - &PrimitiveArray, - ) -> common_arrow::arrow::error::Result, - ) -> Result { - let array = Arc::new(operator(self.downcast_ref(), rhs.downcast_ref())?) as ArrayRef; + fn comparison(&self, rhs: &DataArray, op: Operator) -> Result { + let (lhs, rhs) = (self.array.as_ref(), rhs.array.as_ref()); + let array = Arc::new(compare(lhs, rhs, op)?) as ArrayRef; Ok(array.into()) } -} - -macro_rules! impl_eq_missing { - ($self:ident, $rhs:ident) => {{ - match ($self.null_count(), $rhs.null_count()) { - (0, 0) => $self - .into_no_null_iter() - .zip($rhs.into_no_null_iter()) - .map(|(opt_a, opt_b)| opt_a == opt_b) - .collect(), - (_, _) => $self - .downcast_iter() - .zip($rhs.downcast_iter()) - .map(|(opt_a, opt_b)| opt_a == opt_b) - .collect(), - } - }}; -} -macro_rules! apply { - ($self:expr, $f:expr) => {{ - if $self.null_count() == 0 { - $self.into_no_null_iter().map($f).collect() - } else { - $self.downcast_iter().map(|opt_v| opt_v.map($f)).collect() - } - }}; + fn comparison_scalar(&self, rhs: T::Native, op: Operator) -> Result { + let array = Arc::new(primitive_compare_scalar(self.as_ref(), rhs, op)?) as ArrayRef; + Ok(array.into()) + } } -macro_rules! impl_cmp_numeric_utf8 { - ($self:ident, $rhs:ident, $op:ident, $kop:ident, $operand:tt) => {{ - // broadcast - if $rhs.len() == 1 { +macro_rules! impl_cmp_common { + ($self:ident, $rhs:ident, $kop:ident, $neg_func:tt) => {{ + if $self.len() == $rhs.len() { + $self.comparison($rhs, Operator::$kop) + } else if $rhs.len() == 1 { if let Some(value) = $rhs.get(0) { - $self.$op(value) + $self.comparison_scalar(value, Operator::$kop) } else { Ok(DFBooleanArray::full(false, $self.len())) } } else if $self.len() == 1 { - if let Some(value) = $self.get(0) { - let f = |c| value $operand c; - Ok(apply! {$rhs, f}) - } else { - Ok(DFBooleanArray::full(false, $rhs.len())) - } - } else if $self.len() == $rhs.len() { - $self.comparison($rhs, comparison::$kop) + $rhs.$neg_func($self) } else { - Ok(apply_operand_on_array_by_iter!($self, $rhs, $operand)) + unreachable!() } }}; } @@ -169,307 +141,168 @@ where T: DFNumericType, T::Native: NumComp, { - fn eq_missing(&self, rhs: &DataArray) -> Result { - Ok(impl_eq_missing!(self, rhs)) - } - fn eq(&self, rhs: &DataArray) -> Result { - impl_cmp_numeric_utf8! {self, rhs, eq, eq, ==} + impl_cmp_common! {self, rhs, Eq, eq} } fn neq(&self, rhs: &DataArray) -> Result { - impl_cmp_numeric_utf8! {self, rhs, neq, neq,!=} + impl_cmp_common! {self, rhs, Neq, neq} } fn gt(&self, rhs: &DataArray) -> Result { - impl_cmp_numeric_utf8! {self, rhs, gt,gt, >} + impl_cmp_common! {self, rhs, Gt, lt_eq} } fn gt_eq(&self, rhs: &DataArray) -> Result { - impl_cmp_numeric_utf8! {self, rhs, gt_eq, gt_eq, >=} + impl_cmp_common! {self, rhs, GtEq, lt} } fn lt(&self, rhs: &DataArray) -> Result { - impl_cmp_numeric_utf8! {self, rhs, lt, lt, <} + impl_cmp_common! {self, rhs, Lt, gt_eq} } fn lt_eq(&self, rhs: &DataArray) -> Result { - impl_cmp_numeric_utf8! {self, rhs, lt_eq, lt_eq, <=} + impl_cmp_common! {self, rhs, LtEq, gt} } } -macro_rules! impl_cmp_bool { - ($self:ident, $rhs:ident, $operand:tt) => {{ - // broadcast - if $rhs.len() == 1 { - if let Some(value) = $rhs.get(0) { - match value { - true => Ok($self.clone()), - false => $self.not(), - } - } else { - Ok(DFBooleanArray::full(false, $self.len())) - } - } else if $self.len() == 1 { - if let Some(value) = $self.get(0) { - match value { - true => Ok($rhs.clone()), - false => $rhs.not(), - } - } else { - Ok(DFBooleanArray::full(false, $rhs.len())) - } - } else { - Ok(apply_operand_on_array_by_iter!($self, $rhs, $operand)) - } - }}; -} +impl DFBooleanArray { + /// First ensure that the Arrays of lhs and rhs match and then iterates over the Arrays and applies + /// the comparison operator. + fn comparison(&self, rhs: &DFBooleanArray, op: Operator) -> Result { + let (lhs, rhs) = (self.array.as_ref(), rhs.array.as_ref()); + let array = Arc::new(compare(lhs, rhs, op)?) as ArrayRef; + Ok(array.into()) + } -impl ArrayCompare<&DFBooleanArray> for DFBooleanArray { - fn eq_missing(&self, rhs: &DFBooleanArray) -> Result { - Ok(impl_eq_missing!(self, rhs)) + fn comparison_scalar(&self, rhs: bool, op: Operator) -> Result { + let array = Arc::new(boolean_compare_scalar(self.as_ref(), rhs, op)?) as ArrayRef; + Ok(array.into()) } +} +impl ArrayCompare<&DFBooleanArray> for DFBooleanArray { fn eq(&self, rhs: &DFBooleanArray) -> Result { - impl_cmp_bool! {self, rhs, == } + impl_cmp_common! {self, rhs, Eq, eq} } fn neq(&self, rhs: &DFBooleanArray) -> Result { - impl_cmp_bool! {self, rhs, != } + impl_cmp_common! {self, rhs, Neq, neq} } fn gt(&self, rhs: &DFBooleanArray) -> Result { - impl_cmp_bool! {self, rhs, > } + impl_cmp_common! {self, rhs, Gt, lt_eq} } fn gt_eq(&self, rhs: &DFBooleanArray) -> Result { - impl_cmp_bool! {self, rhs, >= } + impl_cmp_common! {self, rhs, GtEq, lt} } fn lt(&self, rhs: &DFBooleanArray) -> Result { - impl_cmp_bool! {self, rhs, < } + impl_cmp_common! {self, rhs, Lt, gt_eq} } fn lt_eq(&self, rhs: &DFBooleanArray) -> Result { - impl_cmp_bool! {self, rhs, <= } + impl_cmp_common! {self, rhs, LtEq, gt} } } impl DFUtf8Array { - fn comparison( - &self, - rhs: &DFUtf8Array, - operator: impl Fn( - &StringArray, - &StringArray, - ) -> common_arrow::arrow::error::Result, - ) -> Result { - let arr = operator(self.downcast_ref(), rhs.downcast_ref())?; - Ok(DFBooleanArray::from_arrow_array(arr)) + fn comparison(&self, rhs: &DFUtf8Array, op: Operator) -> Result { + let (lhs, rhs) = (self.array.as_ref(), rhs.array.as_ref()); + let array = Arc::new(compare(lhs, rhs, op)?) as ArrayRef; + Ok(array.into()) + } + + fn comparison_scalar(&self, rhs: &str, op: Operator) -> Result { + let array = Arc::new(utf8_compare_scalar(self.as_ref(), rhs, op)) as ArrayRef; + Ok(array.into()) + } + + // pub fn like_utf8(lhs: &Utf8Array, rhs: &Utf8Array) + fn like(&self, rhs: &DFUtf8Array) -> Result { + let array = like::like_utf8(self.downcast_ref(), rhs.downcast_ref())?; + Ok(DFBooleanArray::from_arrow_array(array)) + } + + fn like_scalar(&self, rhs: &str) -> Result { + let array = like::like_utf8_scalar(self.downcast_ref(), rhs)?; + Ok(DFBooleanArray::from_arrow_array(array)) + } + + fn nlike(&self, rhs: &DFUtf8Array) -> Result { + let array = like::nlike_utf8(self.downcast_ref(), rhs.downcast_ref())?; + Ok(DFBooleanArray::from_arrow_array(array)) + } + + fn nlike_scalar(&self, rhs: &str) -> Result { + let array = like::nlike_utf8_scalar(self.downcast_ref(), rhs)?; + Ok(DFBooleanArray::from_arrow_array(array)) } } macro_rules! impl_like_utf8 { - ($self:ident, $rhs:ident, $op:ident, $kop:ident) => {{ + ($self:ident, $rhs:ident, $op:ident, $scalar_op:ident) => {{ // broadcast if $rhs.len() == 1 { if let Some(value) = $rhs.get(0) { - $self.$op(value) + $self.$scalar_op(value) } else { Ok(DFBooleanArray::full(false, $self.len())) } } else if $self.len() == 1 { if let Some(value) = $self.get(0) { - $rhs.$op(value) + let it = (0..$rhs.len()).map(|_| value); + let left = DFUtf8Array::new_from_iter(it); + left.$op($rhs) } else { Ok(DFBooleanArray::full(false, $rhs.len())) } } else { - $self.comparison($rhs, comparison::$kop) + $self.$op($rhs) } }}; } impl ArrayCompare<&DFUtf8Array> for DFUtf8Array { - fn eq_missing(&self, rhs: &DFUtf8Array) -> Result { - Ok(impl_eq_missing!(self, rhs)) - } - fn eq(&self, rhs: &DFUtf8Array) -> Result { - impl_cmp_numeric_utf8! {self, rhs, eq, eq_utf8, ==} + impl_cmp_common! {self, rhs, Eq, eq} } fn neq(&self, rhs: &DFUtf8Array) -> Result { - impl_cmp_numeric_utf8! {self, rhs, neq, neq_utf8, !=} + impl_cmp_common! {self, rhs, Neq, neq} } fn gt(&self, rhs: &DFUtf8Array) -> Result { - impl_cmp_numeric_utf8! {self, rhs, gt, gt_utf8, >} + impl_cmp_common! {self, rhs, Gt, lt_eq} } fn gt_eq(&self, rhs: &DFUtf8Array) -> Result { - impl_cmp_numeric_utf8! {self, rhs, gt_eq, gt_eq_utf8, >=} + impl_cmp_common! {self, rhs, GtEq, lt} } fn lt(&self, rhs: &DFUtf8Array) -> Result { - impl_cmp_numeric_utf8! {self, rhs, lt, lt_utf8, <} + impl_cmp_common! {self, rhs, Lt, gt_eq} } fn lt_eq(&self, rhs: &DFUtf8Array) -> Result { - impl_cmp_numeric_utf8! {self, rhs, lt_eq, lt_eq_utf8, <=} + impl_cmp_common! {self, rhs, LtEq, gt} } fn like(&self, rhs: &DFUtf8Array) -> Result { - impl_like_utf8! {self, rhs, like, like_utf8} + impl_like_utf8! {self, rhs, like, like_scalar} } fn nlike(&self, rhs: &DFUtf8Array) -> Result { - impl_like_utf8! {self, rhs, nlike, nlike_utf8} + impl_like_utf8! {self, rhs, nlike, nlike_scalar} } } impl ArrayCompare<&DFNullArray> for DFNullArray {} -impl ArrayCompare<&DFBinaryArray> for DFBinaryArray {} -impl ArrayCompare<&DFStructArray> for DFStructArray {} - -pub trait NumComp: Num + NumCast + PartialOrd {} - -impl NumComp for f32 {} -impl NumComp for f64 {} -impl NumComp for i8 {} -impl NumComp for i16 {} -impl NumComp for i32 {} -impl NumComp for i64 {} -impl NumComp for u8 {} -impl NumComp for u16 {} -impl NumComp for u32 {} -impl NumComp for u64 {} - -impl ArrayCompare for DataArray -where - T: DFNumericType, - T::Native: NumCast, - Rhs: NumComp + ToPrimitive, -{ - fn eq_missing(&self, rhs: Rhs) -> Result { - self.eq(rhs) - } - - fn eq(&self, rhs: Rhs) -> Result { - let rhs = NumCast::from(rhs); - match rhs { - Some(v) => { - let arr = eq_scalar(self.downcast_ref(), v)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - None => Ok(DFBooleanArray::full(false, self.len())), - } - } - - fn neq(&self, rhs: Rhs) -> Result { - let rhs = NumCast::from(rhs); - match rhs { - Some(v) => { - let arr = neq_scalar(self.downcast_ref(), v)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - None => Ok(DFBooleanArray::full(false, self.len())), - } - } - - fn gt(&self, rhs: Rhs) -> Result { - let rhs = NumCast::from(rhs); - match rhs { - Some(v) => { - let arr = gt_scalar(self.downcast_ref(), v)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - None => Ok(DFBooleanArray::full(false, self.len())), - } - } - - fn gt_eq(&self, rhs: Rhs) -> Result { - let rhs = NumCast::from(rhs); - match rhs { - Some(v) => { - let arr = gt_eq_scalar(self.downcast_ref(), v)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - None => Ok(DFBooleanArray::full(false, self.len())), - } - } - - fn lt(&self, rhs: Rhs) -> Result { - let rhs = NumCast::from(rhs); - - match rhs { - Some(v) => { - let arr = lt_scalar(self.downcast_ref(), v)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - None => Ok(DFBooleanArray::full(false, self.len())), - } - } - - fn lt_eq(&self, rhs: Rhs) -> Result { - let rhs = NumCast::from(rhs); - - match rhs { - Some(v) => { - let arr = lt_eq_scalar(self.downcast_ref(), v)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - None => Ok(DFBooleanArray::full(false, self.len())), - } - } -} - -impl ArrayCompare<&str> for DFUtf8Array { - fn eq_missing(&self, rhs: &str) -> Result { - self.eq(rhs) - } - - fn eq(&self, rhs: &str) -> Result { - let arr = eq_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - - fn neq(&self, rhs: &str) -> Result { - let arr = neq_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - - fn gt(&self, rhs: &str) -> Result { - let arr = gt_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - - fn gt_eq(&self, rhs: &str) -> Result { - let arr = gt_eq_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - - fn lt(&self, rhs: &str) -> Result { - let arr = lt_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - - fn lt_eq(&self, rhs: &str) -> Result { - let arr = lt_eq_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } - - fn like(&self, rhs: &str) -> Result { - let arr = like_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } +impl ArrayCompare<&DFBinaryArray> for DFBinaryArray {} - fn nlike(&self, rhs: &str) -> Result { - let arr = nlike_utf8_scalar(self.downcast_ref(), rhs)?; - Ok(DFBooleanArray::from_arrow_array(arr)) - } -} +impl ArrayCompare<&DFStructArray> for DFStructArray {} macro_rules! impl_cmp_numeric_utf8_list { ($self:ident, $rhs:ident, $cmp_method:ident) => {{ @@ -504,10 +337,6 @@ macro_rules! impl_cmp_numeric_utf8_list { } impl ArrayCompare<&DFListArray> for DFListArray { - fn eq_missing(&self, rhs: &DFListArray) -> Result { - Ok(impl_cmp_numeric_utf8_list!(self, rhs, series_equal_missing)) - } - fn eq(&self, rhs: &DFListArray) -> Result { Ok(impl_cmp_numeric_utf8_list!(self, rhs, series_equal)) } @@ -562,6 +391,9 @@ impl ArrayEqualElement for DFUtf8Array { } impl ArrayEqualElement for DFListArray {} + impl ArrayEqualElement for DFNullArray {} + impl ArrayEqualElement for DFStructArray {} + impl ArrayEqualElement for DFBinaryArray {} diff --git a/common/datavalues/src/arrays/data_array.rs b/common/datavalues/src/arrays/data_array.rs index 648e580952863..bff363d3f1891 100644 --- a/common/datavalues/src/arrays/data_array.rs +++ b/common/datavalues/src/arrays/data_array.rs @@ -5,23 +5,19 @@ use std::convert::TryFrom; use std::fmt::Formatter; use std::marker::PhantomData; -use std::ops::BitAnd; use std::sync::Arc; use common_arrow::arrow::array as arrow_array; use common_arrow::arrow::array::*; use common_arrow::arrow::bitmap::Bitmap; -use common_arrow::arrow::buffer::Buffer; -use common_arrow::arrow::datatypes::IntervalUnit; -use common_arrow::arrow::datatypes::TimeUnit; +use common_arrow::arrow::compute::aggregate; +use common_arrow::arrow::trusted_len::TrustedLen; use common_exception::ErrorCode; use common_exception::Result; use crate::data_df_type::*; -use crate::series::IntoSeries; -use crate::series::Series; -use crate::series::SeriesTrait; -use crate::vec::AlignedVec; +use crate::prelude::*; +use crate::utils::CustomIterTools; use crate::DataType; use crate::DataValue; @@ -68,16 +64,8 @@ impl DataArray { } /// Get the null count and the buffer of bits representing null values - pub fn null_bits(&self) -> (usize, Option) { - let data = self.array.data(); - - ( - data.null_count(), - data.null_bitmap().as_ref().map(|bitmap| { - let buff = bitmap.buffer_ref(); - buff.clone() - }), - ) + pub fn null_bits(&self) -> (usize, &Option) { + (self.array.null_count(), self.array.validity()) } pub fn limit(&self, num_elements: usize) -> Self { @@ -85,12 +73,12 @@ impl DataArray { } pub fn get_array_memory_size(&self) -> usize { - self.array.get_array_memory_size() + aggregate::estimated_bytes_size(self.array.as_ref()) } pub fn slice(&self, offset: usize, length: usize) -> Self { - let array = self.array.slice(offset, length); - array.into() + let array = Arc::from(self.array.slice(offset, length)); + Self::from(array) } /// Unpack a array to the same physical type. @@ -149,7 +137,7 @@ where T: DFDataType // TODO: insert types match T::data_type() { - DataType::Utf8 => downcast_and_pack!(StringArray, Utf8), + DataType::Utf8 => downcast_and_pack!(LargeUtf8Array, Utf8), DataType::Boolean => downcast_and_pack!(BooleanArray, Boolean), DataType::UInt8 => downcast_and_pack!(UInt8Array, UInt8), DataType::UInt16 => downcast_and_pack!(UInt16Array, UInt16), @@ -161,40 +149,17 @@ where T: DFDataType DataType::Int64 => downcast_and_pack!(Int64Array, Int64), DataType::Float32 => downcast_and_pack!(Float32Array, Float32), DataType::Float64 => downcast_and_pack!(Float64Array, Float64), - DataType::Date32 => downcast_and_pack!(Date32Array, Date32), - DataType::Date64 => downcast_and_pack!(Date64Array, Date64), - - DataType::Timestamp(TimeUnit::Second, _) => { - downcast_and_pack!(TimestampSecondArray, TimestampSecond) - } - DataType::Timestamp(TimeUnit::Millisecond, _) => { - downcast_and_pack!(TimestampMillisecondArray, TimestampMillisecond) - } - DataType::Timestamp(TimeUnit::Microsecond, _) => { - downcast_and_pack!(TimestampMicrosecondArray, TimestampMicrosecond) - } - DataType::Timestamp(TimeUnit::Nanosecond, _) => { - downcast_and_pack!(TimestampNanosecondArray, TimestampNanosecond) - } - - DataType::Interval(IntervalUnit::YearMonth) => { - downcast_and_pack!(IntervalYearMonthArray, IntervalYearMonth) - } - - DataType::Interval(IntervalUnit::DayTime) => { - downcast_and_pack!(IntervalDayTimeArray, IntervalDayTime) - } DataType::Binary => { - downcast_and_pack!(BinaryArray, Binary) + downcast_and_pack!(LargeBinaryArray, Binary) } DataType::List(fs) => { - let list_array = &*(arr as *const dyn Array as *const ListArray); + let list_array = &*(arr as *const dyn Array as *const LargeListArray); let value = match list_array.is_null(index) { true => None, false => { - let nested_array = list_array.value(index); + let nested_array: Arc = Arc::from(list_array.value(index)); let series = nested_array.into_series(); let scalar_vec = (0..series.len()) .map(|i| series.try_get(i)) @@ -208,10 +173,10 @@ where T: DFDataType DataType::Struct(_) => { let struct_array = &*(arr as *const dyn Array as *const StructArray); - let nested_array = struct_array.column(index); - let series = nested_array.clone().into_series(); + let nested_array = struct_array.values()[index].clone(); + let series = nested_array.into_series(); - let scalar_vec = (0..nested_array.len()) + let scalar_vec = (0..series.len()) .map(|i| series.try_get(i)) .collect::>>()?; Ok(DataValue::Struct(scalar_vec)) @@ -223,59 +188,23 @@ where T: DFDataType ))), } } - - // Apply BitAnd with the null masks and generate a new ArrayData - pub fn apply_null_mask(&self, mask: impl AsRef<[u8]>) -> Result { - let mask = mask.as_ref(); - if mask.len() != self.len() { - return Err(ErrorCode::BadDataArrayLength(format!( - "cannot apply null mask, size not matched, got: {}, expect: {}", - mask.len(), - self.len(), - ))); - } - let mut builder = BooleanBufferBuilder::new(mask.len()); - for b in mask.iter() { - builder.append(*b > 0); - } - let buffer = builder.finish(); - let data = self.array.data(); - let bitmap = Bitmap::from(buffer); - - let bitmap_and = if let Some(b) = data.null_bitmap() { - b.bitand(&bitmap)? - } else { - bitmap - }; - - let array_data = ArrayData::new( - T::data_type().to_arrow(), - data.len(), - None, - Some(bitmap_and.into_buffer()), - data.offset(), - data.buffers().to_owned(), - data.child_data().to_owned(), - ); - Ok(make_array(array_data).into()) - } } impl DataArray where T: DFPrimitiveType { /// Create a new DataArray by taking ownership of the AlignedVec. This operation is zero copy. - pub fn new_from_aligned_vec(v: AlignedVec) -> Self { - let array = v.into_primitive_array::(None); + pub fn new_from_aligned_vec(values: AlignedVec) -> Self { + let array = to_primitive::(values, None); Self::new(Arc::new(array)) } /// Nullify values in slice with an existing null bitmap pub fn new_from_owned_with_null_bitmap( values: AlignedVec, - buffer: Option, + validity: Option, ) -> Self { - let array = values.into_primitive_array::(buffer); + let array = to_primitive::(values, validity); Self::new(Arc::new(array)) } @@ -287,17 +216,20 @@ where T: DFPrimitiveType &self, ) -> impl Iterator + '_ + Send + Sync + ExactSizeIterator + DoubleEndedIterator { - self.downcast_ref().values().iter() + self.downcast_ref().values().as_slice().iter() } - #[allow(clippy::wrong_self_convention)] pub fn into_no_null_iter( &self, - ) -> impl Iterator + '_ + Send + Sync + ExactSizeIterator + DoubleEndedIterator - { + ) -> impl Iterator + + '_ + + Send + + Sync + + ExactSizeIterator + + DoubleEndedIterator + + TrustedLen { // .copied was significantly slower in benchmark, next call did not inline? - #[allow(clippy::map_clone)] - self.data_views().map(|v| *v) + self.data_views().copied().trust_my_length(self.len()) } } @@ -316,6 +248,12 @@ impl From for DataArray { } } +impl From> for DataArray { + fn from(array: Box) -> Self { + Self::new(Arc::from(array)) + } +} + impl From<&arrow_array::ArrayRef> for DataArray { fn from(array: &arrow_array::ArrayRef) -> Self { Self::new(array.clone()) @@ -335,3 +273,11 @@ where T: DFDataType write!(f, "DataArray<{:?}>", self.data_type()) } } + +#[inline] +pub fn to_primitive( + values: AlignedVec, + validity: Option, +) -> PrimitiveArray { + PrimitiveArray::from_data(T::data_type().to_arrow(), values.into(), validity) +} diff --git a/common/datavalues/src/arrays/data_array_test.rs b/common/datavalues/src/arrays/data_array_test.rs deleted file mode 100644 index 52dd9e7fe7f46..0000000000000 --- a/common/datavalues/src/arrays/data_array_test.rs +++ /dev/null @@ -1,18 +0,0 @@ -// Copyright 2020-2021 The Datafuse Authors. -// -// SPDX-License-Identifier: Apache-2.0. - -use common_exception::Result; - -use crate::prelude::*; - -#[test] -fn test_data_array() -> Result<()> { - let array = DFUInt16Array::new_from_iter(1..10); - assert_eq!(array.null_count(), 0); - - let mask: Vec<_> = (1..10).map(|c| (c % 3 > 0) as u8).collect(); - let array = array.apply_null_mask(&mask).unwrap(); - assert_eq!(array.null_count(), 3); - Ok(()) -} diff --git a/common/datavalues/src/arrays/kernels/iterator.rs b/common/datavalues/src/arrays/kernels/iterator.rs index c09b1efc9eeb7..7c1ee724c4d90 100644 --- a/common/datavalues/src/arrays/kernels/iterator.rs +++ b/common/datavalues/src/arrays/kernels/iterator.rs @@ -2,14 +2,12 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::ListArray; -use common_arrow::arrow::array::StringArray; +use common_arrow::arrow::array::*; +use common_arrow::arrow::trusted_len::TrustedLen; -use crate::prelude::DataArray; -use crate::series::IntoSeries; +use crate::prelude::*; use crate::series::Series; +use crate::utils::CustomIterTools; use crate::DFBooleanArray; use crate::DFListArray; use crate::DFNumericType; @@ -18,8 +16,11 @@ use crate::DFUtf8Array; /// A `DFIterator` is an iterator over a `DFArray` which contains DF types. A `DFIterator` /// must implement `DoubleEndedIterator`. pub trait DFIterator: DoubleEndedIterator + Send + Sync {} +unsafe impl<'a, I> TrustedLen for Box + 'a> {} + /// Implement DFIterator for every iterator that implements the needed traits. -impl DFIterator for T where T: DoubleEndedIterator + Send + Sync {} +impl DFIterator for T where T: ExactSizeIterator + DoubleEndedIterator + Send + Sync + TrustedLen +{} impl<'a, T> IntoIterator for &'a DataArray where T: DFNumericType @@ -28,7 +29,11 @@ where T: DFNumericType type IntoIter = Box + 'a>; fn into_iter(self) -> Self::IntoIter { - Box::new(self.downcast_iter()) + Box::new( + self.downcast_iter() + .map(|x| x.copied()) + .trust_my_length(self.len()), + ) } } /// The no null iterator for a BooleanArray @@ -42,7 +47,7 @@ impl<'a> IntoIterator for &'a DFBooleanArray { type Item = Option; type IntoIter = Box + 'a>; fn into_iter(self) -> Self::IntoIter { - Box::new(self.downcast_iter()) + Box::new(self.downcast_iter().trust_my_length(self.len())) } } @@ -90,7 +95,6 @@ impl<'a> DoubleEndedIterator for BoolIterNoNull<'a> { } impl DFBooleanArray { - #[allow(clippy::wrong_self_convention)] pub fn into_no_null_iter( &self, ) -> impl Iterator + '_ + Send + Sync + DoubleEndedIterator { @@ -107,14 +111,14 @@ impl<'a> IntoIterator for &'a DFUtf8Array { } pub struct Utf8IterNoNull<'a> { - array: &'a StringArray, + array: &'a LargeUtf8Array, current: usize, current_end: usize, } impl<'a> Utf8IterNoNull<'a> { /// create a new iterator - pub fn new(array: &'a StringArray) -> Self { + pub fn new(array: &'a LargeUtf8Array) -> Self { Utf8IterNoNull { array, current: 0, @@ -159,7 +163,6 @@ impl<'a> DoubleEndedIterator for Utf8IterNoNull<'a> { impl<'a> ExactSizeIterator for Utf8IterNoNull<'a> {} impl DFUtf8Array { - #[allow(clippy::wrong_self_convention)] pub fn into_no_null_iter<'a>( &'a self, ) -> impl Iterator + '_ + Send + Sync + DoubleEndedIterator { @@ -171,19 +174,19 @@ impl<'a> IntoIterator for &'a DFListArray { type Item = Option; type IntoIter = Box + 'a>; fn into_iter(self) -> Self::IntoIter { - Box::new(self.downcast_iter()) + Box::new(self.downcast_iter().trust_my_length(self.len())) } } pub struct ListIterNoNull<'a> { - array: &'a ListArray, + array: &'a LargeListArray, current: usize, current_end: usize, } impl<'a> ListIterNoNull<'a> { /// create a new iterator - pub fn new(array: &'a ListArray) -> Self { + pub fn new(array: &'a LargeListArray) -> Self { ListIterNoNull { array, current: 0, @@ -201,7 +204,7 @@ impl<'a> Iterator for ListIterNoNull<'a> { } else { let old = self.current; self.current += 1; - let array = unsafe { self.array.value_unchecked(old) }; + let array: ArrayRef = Arc::from(unsafe { self.array.value_unchecked(old) }); Some(array.into_series()) } } @@ -221,7 +224,8 @@ impl<'a> DoubleEndedIterator for ListIterNoNull<'a> { } else { self.current_end -= 1; - let array = unsafe { self.array.value_unchecked(self.current_end) }; + let array: ArrayRef = + Arc::from(unsafe { self.array.value_unchecked(self.current_end) }); Some(array.into_series()) } } @@ -231,7 +235,6 @@ impl<'a> DoubleEndedIterator for ListIterNoNull<'a> { impl<'a> ExactSizeIterator for ListIterNoNull<'a> {} impl DFListArray { - #[allow(clippy::wrong_self_convention)] pub fn into_no_null_iter( &self, ) -> impl Iterator + '_ + Send + Sync + DoubleEndedIterator { diff --git a/common/datavalues/src/arrays/kernels/take.rs b/common/datavalues/src/arrays/kernels/take.rs index 47509d5b84f69..894d380a7fd75 100644 --- a/common/datavalues/src/arrays/kernels/take.rs +++ b/common/datavalues/src/arrays/kernels/take.rs @@ -2,151 +2,51 @@ // // SPDX-License-Identifier: Apache-2.0. -use std::mem; use std::sync::Arc; -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::ArrayData; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; -use common_arrow::arrow::array::StringBuilder; -use common_arrow::arrow::array::UInt32Array; -use common_arrow::arrow::buffer::MutableBuffer; -use common_arrow::arrow::datatypes::DataType as ArrowDataType; +use common_arrow::arrow::array::*; +use common_arrow::arrow::buffer::Buffer; -use crate::arrays::IntoTakeRandom; -use crate::arrays::*; -use crate::*; - -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -/// Take kernel for single chunk without nulls and arrow array as index. -pub unsafe fn take_no_null_primitive( - arr: &PrimitiveArray, - indices: &UInt32Array, -) -> Arc> { - assert_eq!(arr.null_count(), 0); - - let data_len = indices.len(); - let array_values = arr.values(); - let index_values = indices.values(); - - let mut av = AlignedVec::::with_capacity_len_aligned(data_len); - av.iter_mut() - .zip(index_values.iter()) - .for_each(|(num, idx)| { - *num = *array_values.get_unchecked(*idx as usize); - }); - - let nulls = indices.data_ref().null_buffer().cloned(); - let arr = av.into_primitive_array::(nulls); - Arc::new(arr) -} +use crate::prelude::*; /// # Safety /// Note this doesn't do any bound checking, for performance reason. /// Take kernel for single chunk without nulls and an iterator as index. -pub unsafe fn take_no_null_primitive_iter_unchecked< - T: DFNumericType, - I: IntoIterator, ->( - arr: &PrimitiveArray, - indices: I, -) -> Arc> { - assert_eq!(arr.null_count(), 0); - let indices_iter = indices.into_iter(); - let data_len = indices_iter.size_hint().0; - let array_values = arr.values(); - - let mut av = AlignedVec::::with_capacity_len_aligned(data_len); - - av.iter_mut().zip(indices_iter).for_each(|(num, idx)| { - *num = *array_values.get_unchecked(idx); - }); - let arr = av.into_primitive_array::(None); - Arc::new(arr) -} - -/// Take kernel for single chunk without nulls and an iterator as index that does bound checks. -pub fn take_no_null_primitive_iter>( - arr: &PrimitiveArray, - indices: I, -) -> Arc> { - assert_eq!(arr.null_count(), 0); - - let array_values = arr.values(); - - let av = indices - .into_iter() - .map(|idx| array_values[idx]) - .collect::>(); - let arr = av.into_primitive_array(None); - - Arc::new(arr) -} - -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -/// Take kernel for a single chunk with null values and an iterator as index. pub unsafe fn take_primitive_iter_unchecked>( - arr: &PrimitiveArray, - indices: I, -) -> Arc> { - let array_values = arr.values(); - - let iter = indices.into_iter().map(|idx| { - if arr.is_valid(idx) { - Some(*array_values.get_unchecked(idx)) - } else { - None + arr: &PrimitiveArray, + indices: I, +) -> Arc> { + match arr.null_count() { + 0 => { + let array_values = arr.values().as_slice(); + let iter = indices + .into_iter() + .map(|idx| *array_values.get_unchecked(idx)); + + let values = Buffer::from_trusted_len_iter_unchecked(iter); + Arc::new(PrimitiveArray::from_data( + T::data_type().to_arrow(), + values, + None, + )) } - }); - let arr = PrimitiveArray::from_trusted_len_iter(iter); - - Arc::new(arr) -} - -/// Take kernel for a single chunk with null values and an iterator as index that does bound checks. -pub fn take_primitive_iter>( - arr: &PrimitiveArray, - indices: I, -) -> Arc> { - let array_values = arr.values(); - - let arr = indices - .into_iter() - .map(|idx| { - if arr.is_valid(idx) { - Some(array_values[idx]) - } else { - None - } - }) - .collect(); - - Arc::new(arr) -} - -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -/// Take kernel for a single chunk without nulls and an iterator that can produce None values. -/// This is used in join operations. -pub unsafe fn take_no_null_primitive_opt_iter_unchecked< - T: DFNumericType, - I: IntoIterator>, ->( - arr: &PrimitiveArray, - indices: I, -) -> Arc> { - let array_values = arr.values(); - - let iter = indices - .into_iter() - .map(|opt_idx| opt_idx.map(|idx| *array_values.get_unchecked(idx))); - let arr = PrimitiveArray::from_trusted_len_iter(iter); + _ => { + let array_values = arr.values(); + + let arr = indices + .into_iter() + .map(|idx| { + if arr.is_valid(idx) { + Some(array_values[idx]) + } else { + None + } + }) + .collect(); - Arc::new(arr) + Arc::new(arr) + } + } } /// # Safety @@ -157,106 +57,70 @@ pub unsafe fn take_primitive_opt_iter_unchecked< T: DFNumericType, I: IntoIterator>, >( - arr: &PrimitiveArray, - indices: I, -) -> Arc> { - let array_values = arr.values(); - - let iter = indices.into_iter().map(|opt_idx| { - opt_idx.and_then(|idx| { - if arr.is_valid(idx) { - Some(*array_values.get_unchecked(idx)) - } else { - None - } - }) - }); - let arr = PrimitiveArray::from_trusted_len_iter(iter); - - Arc::new(arr) -} - -/// Take kernel for multiple chunks. We directly return a DataArray because that path chooses the fastest collection path. -pub fn take_primitive_iter_n_arrays>( - ca: &DataArray, + arr: &PrimitiveArray, indices: I, -) -> DataArray { - let taker = ca.take_rand(); - indices.into_iter().map(|idx| taker.get(idx)).collect() -} +) -> Arc> { + match arr.null_count() { + 0 => { + let array_values = arr.values(); -/// Take kernel for multiple chunks where an iterator can produce None values. -/// Used in join operations. We directly return a DataArray because that path chooses the fastest collection path. -pub fn take_primitive_opt_iter_n_arrays>>( - ca: &DataArray, - indices: I, -) -> DataArray { - let taker = ca.take_rand(); - indices - .into_iter() - .map(|opt_idx| opt_idx.and_then(|idx| taker.get(idx))) - .collect() -} + let iter = indices + .into_iter() + .map(|opt_idx| opt_idx.map(|idx| *array_values.get_unchecked(idx))); + let arr = PrimitiveArray::from_trusted_len_iter_unchecked(iter); -/// Take kernel for single chunk without nulls and an iterator as index that does bound checks. -pub fn take_no_null_bool_iter>( - arr: &BooleanArray, - indices: I, -) -> Arc { - debug_assert_eq!(arr.null_count(), 0); + Arc::new(arr) + } + _ => { + let array_values = arr.values(); - let iter = indices.into_iter().map(|idx| Some(arr.value(idx))); + let iter = indices.into_iter().map(|opt_idx| { + opt_idx.and_then(|idx| { + if arr.is_valid(idx) { + Some(*array_values.get_unchecked(idx)) + } else { + None + } + }) + }); + let arr = PrimitiveArray::from_trusted_len_iter_unchecked(iter); - Arc::new(iter.collect()) + Arc::new(arr) + } + } } /// # Safety /// Note this doesn't do any bound checking, for performance reason. /// Take kernel for single chunk without nulls and an iterator as index. -pub unsafe fn take_no_null_bool_iter_unchecked>( +pub unsafe fn take_bool_iter_unchecked>( arr: &BooleanArray, indices: I, ) -> Arc { - debug_assert_eq!(arr.null_count(), 0); - let iter = indices - .into_iter() - .map(|idx| Some(arr.value_unchecked(idx))); - - Arc::new(iter.collect()) -} + match arr.null_count() { + 0 => { + let iter = indices.into_iter().map(|idx| { + if arr.is_null(idx) { + None + } else { + Some(arr.value(idx)) + } + }); -/// Take kernel for single chunk and an iterator as index that does bound checks. -pub fn take_bool_iter>( - arr: &BooleanArray, - indices: I, -) -> Arc { - let iter = indices.into_iter().map(|idx| { - if arr.is_null(idx) { - None - } else { - Some(arr.value(idx)) + Arc::new(iter.collect()) } - }); - - Arc::new(iter.collect()) -} + _ => { + let iter = indices.into_iter().map(|idx| { + if arr.is_null(idx) { + None + } else { + Some(arr.value_unchecked(idx)) + } + }); -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -/// Take kernel for single chunk and an iterator as index. -pub unsafe fn take_bool_iter_unchecked>( - arr: &BooleanArray, - indices: I, -) -> Arc { - let iter = indices.into_iter().map(|idx| { - if arr.is_null(idx) { - None - } else { - Some(arr.value_unchecked(idx)) + Arc::new(iter.collect()) } - }); - - Arc::new(iter.collect()) + } } /// # Safety @@ -266,247 +130,82 @@ pub unsafe fn take_bool_opt_iter_unchecked> arr: &BooleanArray, indices: I, ) -> Arc { - let iter = indices.into_iter().map(|opt_idx| { - opt_idx.and_then(|idx| { - if arr.is_null(idx) { - None - } else { - Some(arr.value_unchecked(idx)) - } - }) - }); - - Arc::new(iter.collect()) -} - -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -/// Take kernel for single chunk without null values and an iterator as index that may produce None values. -pub unsafe fn take_no_null_bool_opt_iter_unchecked>>( - arr: &BooleanArray, - indices: I, -) -> Arc { - let iter = indices - .into_iter() - .map(|opt_idx| opt_idx.map(|idx| arr.value_unchecked(idx))); - - Arc::new(iter.collect()) -} + match arr.null_count() { + 0 => { + let iter = indices + .into_iter() + .map(|opt_idx| opt_idx.map(|idx| arr.value_unchecked(idx))); -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -pub unsafe fn take_no_null_utf8_iter_unchecked>( - arr: &StringArray, - indices: I, -) -> Arc { - let iter = indices - .into_iter() - .map(|idx| Some(arr.value_unchecked(idx))); + Arc::new(iter.collect()) + } + _ => { + let iter = indices.into_iter().map(|opt_idx| { + opt_idx.and_then(|idx| { + if arr.is_null(idx) { + None + } else { + Some(arr.value_unchecked(idx)) + } + }) + }); - Arc::new(iter.collect()) + Arc::new(iter.collect()) + } + } } /// # Safety /// Note this doesn't do any bound checking, for performance reason. pub unsafe fn take_utf8_iter_unchecked>( - arr: &StringArray, - indices: I, -) -> Arc { - let iter = indices.into_iter().map(|idx| { - if arr.is_null(idx) { - None - } else { - Some(arr.value_unchecked(idx)) + arr: &LargeUtf8Array, + indices: I, +) -> Arc { + match arr.null_count() { + 0 => { + let iter = indices + .into_iter() + .map(|idx| Some(arr.value_unchecked(idx))); + Arc::new(LargeUtf8Array::from_trusted_len_iter_unchecked(iter)) } - }); - - Arc::new(iter.collect()) -} - -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -pub unsafe fn take_no_null_utf8_opt_iter_unchecked>>( - arr: &StringArray, - indices: I, -) -> Arc { - let iter = indices - .into_iter() - .map(|opt_idx| opt_idx.map(|idx| arr.value_unchecked(idx))); - - Arc::new(iter.collect()) + _ => { + let iter = indices.into_iter().map(|idx| { + if arr.is_null(idx) { + None + } else { + Some(arr.value_unchecked(idx)) + } + }); + Arc::new(LargeUtf8Array::from_trusted_len_iter_unchecked(iter)) + } + } } /// # Safety /// Note this doesn't do any bound checking, for performance reason. pub unsafe fn take_utf8_opt_iter_unchecked>>( - arr: &StringArray, - indices: I, -) -> Arc { - let iter = indices.into_iter().map(|opt_idx| { - opt_idx.and_then(|idx| { - if arr.is_null(idx) { - None - } else { - Some(arr.value_unchecked(idx)) - } - }) - }); - - Arc::new(iter.collect()) -} - -pub fn take_no_null_utf8_iter>( - arr: &StringArray, + arr: &LargeUtf8Array, indices: I, -) -> Arc { - let iter = indices.into_iter().map(|idx| Some(arr.value(idx))); +) -> Arc { + match arr.null_count() { + 0 => { + let iter = indices + .into_iter() + .map(|opt_idx| opt_idx.map(|idx| arr.value_unchecked(idx))); - Arc::new(iter.collect()) -} - -pub fn take_utf8_iter>( - arr: &StringArray, - indices: I, -) -> Arc { - let iter = indices.into_iter().map(|idx| { - if arr.is_null(idx) { - None - } else { - Some(arr.value(idx)) + Arc::new(LargeUtf8Array::from_trusted_len_iter_unchecked(iter)) } - }); - - Arc::new(iter.collect()) -} - -/// # Safety -/// Note this doesn't do any bound checking, for performance reason. -pub unsafe fn take_utf8(arr: &StringArray, indices: &UInt32Array) -> Arc { - let data_len = indices.len(); - - let offset_len_in_bytes = (data_len + 1) * mem::size_of::(); - let mut offset_buf = MutableBuffer::new(offset_len_in_bytes); - offset_buf.resize(offset_len_in_bytes, 0); - let offset_typed = offset_buf.typed_data_mut(); - - let mut length_so_far = 0; - offset_typed[0] = length_so_far; - - let nulls; - - // The required size is yet unknown - // Allocate 2.0 times the expected size. - // where expected size is the length of bytes multiplied by the factor (take_len / current_len) - let mut values_capacity = if arr.len() > 0 { - ((arr.value_data().len() as f32 * 2.0) as usize) / arr.len() * indices.len() as usize - } else { - 0 - }; - - // 16 bytes per string as default alloc - let mut values_buf = AlignedVec::::with_capacity_aligned(values_capacity); - - // both 0 nulls - if arr.null_count() == 0 && indices.null_count() == 0 { - offset_typed - .iter_mut() - .skip(1) - .enumerate() - .for_each(|(idx, offset)| { - let index = indices.value_unchecked(idx) as usize; - let s = arr.value_unchecked(index); - length_so_far += s.len() as i64; - *offset = length_so_far; - - if length_so_far as usize >= values_capacity { - values_buf.reserve(values_capacity); - values_capacity *= 2; - } - - values_buf.extend_from_slice(s.as_bytes()) - }); - nulls = None; - } else if arr.null_count() == 0 { - offset_typed - .iter_mut() - .skip(1) - .enumerate() - .for_each(|(idx, offset)| { - if indices.is_valid(idx) { - let index = indices.value_unchecked(idx) as usize; - let s = arr.value_unchecked(index); - length_so_far += s.len() as i64; - - if length_so_far as usize >= values_capacity { - values_buf.reserve(values_capacity); - values_capacity *= 2; - } - - values_buf.extend_from_slice(s.as_bytes()) - } - *offset = length_so_far; - }); - nulls = indices.data_ref().null_buffer().cloned(); - } else { - let mut builder = StringBuilder::with_capacity(data_len, length_so_far as usize); - - if indices.null_count() == 0 { - (0..data_len).for_each(|idx| { - let index = indices.value_unchecked(idx) as usize; - if arr.is_valid(index) { - let s = arr.value_unchecked(index); - builder.append_value(s).unwrap(); - } else { - builder.append_null().unwrap(); - } - }); - } else { - (0..data_len).for_each(|idx| { - if indices.is_valid(idx) { - let index = indices.value_unchecked(idx) as usize; - - if arr.is_valid(index) { - let s = arr.value_unchecked(index); - builder.append_value(s).unwrap(); + _ => { + let iter = indices.into_iter().map(|opt_idx| { + opt_idx.and_then(|idx| { + if arr.is_null(idx) { + None } else { - builder.append_null().unwrap(); + Some(arr.value_unchecked(idx)) } - } else { - builder.append_null().unwrap(); - } + }) }); - } - - return Arc::new(builder.finish()); - } - - let mut data = ArrayData::builder(ArrowDataType::Utf8) - .len(data_len) - .add_buffer(offset_buf.into()) - .add_buffer(values_buf.into_arrow_buffer()); - if let Some(null_buffer) = nulls { - data = data.null_bit_buffer(null_buffer); - } - Arc::new(StringArray::from(data.build())) -} - -#[cfg(test)] -mod test { - use super::*; - #[test] - fn test_utf8_kernel() { - let s = StringArray::from(vec![Some("foo"), None, Some("bar")]); - unsafe { - let out = take_utf8(&s, &UInt32Array::from(vec![1, 2])); - assert!(out.is_null(0)); - assert!(out.is_valid(1)); - let out = take_utf8(&s, &UInt32Array::from(vec![None, Some(2)])); - assert!(out.is_null(0)); - assert!(out.is_valid(1)); - let out = take_utf8(&s, &UInt32Array::from(vec![None, None])); - assert!(out.is_null(0)); - assert!(out.is_null(1)); + Arc::new(LargeUtf8Array::from_trusted_len_iter_unchecked(iter)) } } } diff --git a/common/datavalues/src/arrays/mod.rs b/common/datavalues/src/arrays/mod.rs index 1aa06d715f1e5..ff9412d0d0e73 100644 --- a/common/datavalues/src/arrays/mod.rs +++ b/common/datavalues/src/arrays/mod.rs @@ -1,25 +1,22 @@ // Copyright 2020-2021 The Datafuse Authors. // // SPDX-License-Identifier: Apache-2.0. -mod arrow_array; mod data_array; -#[cfg(test)] -mod data_array_test; - #[macro_use] mod arithmetic; mod builders; mod comparison; mod kernels; mod ops; +mod trusted_len; mod upstream_traits; pub use arithmetic::*; -pub use arrow_array::*; pub use builders::*; pub use comparison::*; pub use data_array::*; pub use kernels::*; pub use ops::*; +pub use trusted_len::*; pub use upstream_traits::*; diff --git a/common/datavalues/src/arrays/ops/agg.rs b/common/datavalues/src/arrays/ops/agg.rs index 3257170063c1a..fba275543c94c 100644 --- a/common/datavalues/src/arrays/ops/agg.rs +++ b/common/datavalues/src/arrays/ops/agg.rs @@ -3,8 +3,11 @@ // SPDX-License-Identifier: Apache-2.0. use std::fmt::Debug; +use std::ops::Add; -use common_arrow::arrow::compute; +use common_arrow::arrow::compute::aggregate; +use common_arrow::arrow::types::simd::Simd; +use common_arrow::arrow::types::NativeType; use common_exception::ErrorCode; use common_exception::Result; use num::Num; @@ -65,25 +68,28 @@ pub trait ArrayAgg: Debug { impl ArrayAgg for DataArray where T: DFNumericType, - T::Native: PartialOrd + Num + NumCast + Zero + Into, + T::Native: NativeType + Simd + PartialOrd + Num + NumCast + Zero + Into, + ::Simd: Add::Simd> + + aggregate::Sum + + aggregate::SimdOrd, Option: Into, { fn sum(&self) -> Result { - Ok(match compute::sum(self.downcast_ref()) { + Ok(match aggregate::sum(self.downcast_ref()) { Some(x) => x.into(), None => DataValue::from(self.data_type()), }) } fn min(&self) -> Result { - Ok(match compute::min(self.downcast_ref()) { + Ok(match aggregate::min_primitive(self.downcast_ref()) { Some(x) => x.into(), None => DataValue::from(self.data_type()), }) } fn max(&self) -> Result { - Ok(match compute::max(self.downcast_ref()) { + Ok(match aggregate::max_primitive(self.downcast_ref()) { Some(x) => x.into(), None => DataValue::from(self.data_type()), }) @@ -143,14 +149,22 @@ impl ArrayAgg for DFBooleanArray { if self.all_is_null() { return Ok(DataValue::Boolean(None)); } - Ok(min_max_boolean_helper(self, true).into()) + + Ok(match aggregate::min_boolean(self.downcast_ref()) { + Some(x) => x.into(), + None => DataValue::from(self.data_type()), + }) } fn max(&self) -> Result { if self.all_is_null() { return Ok(DataValue::Boolean(None)); } - Ok(min_max_boolean_helper(self, false).into()) + + Ok(match aggregate::max_boolean(self.downcast_ref()) { + Some(x) => x.into(), + None => DataValue::from(self.data_type()), + }) } fn arg_min(&self) -> Result { @@ -202,48 +216,16 @@ impl ArrayAgg for DFBooleanArray { } } -fn min_max_boolean_helper(ca: &DFBooleanArray, min: bool) -> u32 { - ca.downcast_iter().fold(0, |acc: u32, x| match x { - Some(v) => { - let v = v as u32; - if min { - if acc < v { - acc - } else { - v - } - } else if acc > v { - acc - } else { - v - } - } - None => acc, - }) -} - impl ArrayAgg for DFUtf8Array { fn min(&self) -> Result { if self.all_is_null() { return Ok(DataValue::Utf8(None)); } - let d = self.downcast_iter().reduce(|acc, x| match (acc, x) { - (None, _) => x, - (Some(v_acc), Some(v)) => { - if v_acc < v { - acc - } else { - x - } - } - _ => acc, - }); - - if let Some(Some(v)) = d { - return Ok(DataValue::Utf8(Some(v.to_string()))); - } - Ok(DataValue::Utf8(None)) + Ok(match aggregate::min_string(self.downcast_ref()) { + Some(x) => x.into(), + None => DataValue::from(self.data_type()), + }) } fn max(&self) -> Result { @@ -251,22 +233,10 @@ impl ArrayAgg for DFUtf8Array { return Ok(DataValue::Utf8(None)); } - let d = self.downcast_iter().reduce(|acc, x| match (acc, x) { - (None, _) => x, - (Some(v_acc), Some(v)) => { - if v_acc > v { - acc - } else { - x - } - } - _ => acc, - }); - - if let Some(Some(v)) = d { - return Ok(DataValue::Utf8(Some(v.to_string()))); - } - Ok(DataValue::Utf8(None)) + Ok(match aggregate::max_string(self.downcast_ref()) { + Some(x) => x.into(), + None => DataValue::from(self.data_type()), + }) } fn arg_max(&self) -> Result { @@ -315,6 +285,9 @@ impl ArrayAgg for DFUtf8Array { } impl ArrayAgg for DFListArray {} + impl ArrayAgg for DFBinaryArray {} + impl ArrayAgg for DFNullArray {} + impl ArrayAgg for DFStructArray {} diff --git a/common/datavalues/src/arrays/ops/agg_test.rs b/common/datavalues/src/arrays/ops/agg_test.rs index ff184da19652d..f2a2e5a417e09 100644 --- a/common/datavalues/src/arrays/ops/agg_test.rs +++ b/common/datavalues/src/arrays/ops/agg_test.rs @@ -50,8 +50,8 @@ fn test_boolean_array_agg() -> Result<()> { let expected = [ DataValue::UInt32(Some(2)), - DataValue::UInt32(Some(1)), - DataValue::UInt32(Some(0)), + DataValue::Boolean(Some(true)), + DataValue::Boolean(Some(false)), DataValue::Struct(vec![ DataValue::UInt64(Some(1)), DataValue::Boolean(Some(false)), @@ -63,7 +63,7 @@ fn test_boolean_array_agg() -> Result<()> { ]; let len = value.len(); for i in 0..len { - assert_eq!(value[i], expected[i]); + assert_eq!(value[i], expected[i], "in test_{}", i); } Ok(()) } diff --git a/common/datavalues/src/arrays/ops/apply.rs b/common/datavalues/src/arrays/ops/apply.rs index e6f8ea213cca4..3ec5d28ed3da9 100644 --- a/common/datavalues/src/arrays/ops/apply.rs +++ b/common/datavalues/src/arrays/ops/apply.rs @@ -4,15 +4,11 @@ use std::borrow::Cow; use std::sync::Arc; -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; +use common_arrow::arrow::array::*; use crate::arrays::DataArray; +use crate::prelude::*; use crate::utils::NoNull; -use crate::vec::AlignedVec; use crate::*; macro_rules! apply { @@ -93,15 +89,19 @@ where T: DFNumericType F: Fn(T::Native) -> S::Native + Copy, S: DFNumericType, { - let mut av = AlignedVec::::with_capacity_len_aligned(self.len()); + let mut av = AlignedVec::::with_capacity(self.len()); + unsafe { av.set_len(self.len()) } let values = self.as_ref().values(); - av.iter_mut().zip(values.iter()).for_each(|(num, n)| { - *num = f(*n); - }); + av.as_mut_slice() + .iter_mut() + .zip(values.iter()) + .for_each(|(num, n)| { + *num = f(*n); + }); - let (_, buffer) = self.null_bits(); - let array = Arc::new(av.into_primitive_array::(buffer)) as ArrayRef; + let (_, validity) = self.null_bits(); + let array = Arc::new(to_primitive::(av, validity.clone())) as ArrayRef; array.into() } @@ -110,29 +110,38 @@ where T: DFNumericType F: Fn(Option) -> S::Native + Copy, S: DFNumericType, { - let mut av = AlignedVec::::with_capacity_len_aligned(self.len()); + let mut av = AlignedVec::::with_capacity(self.len()); + unsafe { av.set_len(self.len()) } + let array = self.downcast_ref(); - let (_, buffer) = self.null_bits(); - av.iter_mut() + let (_, validity) = self.null_bits(); + + av.as_mut_slice() + .iter_mut() .zip(array.values().iter()) .for_each(|(num, n)| { *num = f(Some(*n)); }); - let array = Arc::new(av.into_primitive_array::(buffer)) as ArrayRef; + let array = Arc::new(to_primitive::(av, validity.clone())) as ArrayRef; array.into() } fn apply(&'a self, f: F) -> Self where F: Fn(T::Native) -> T::Native + Copy { - let mut av = AlignedVec::::with_capacity_len_aligned(self.len()); + let mut av = AlignedVec::::with_capacity(self.len()); + unsafe { av.set_len(self.len()) } + let values = self.as_ref().values(); - av.iter_mut().zip(values.iter()).for_each(|(num, n)| { - *num = f(*n); - }); + av.as_mut_slice() + .iter_mut() + .zip(values.iter()) + .for_each(|(num, n)| { + *num = f(*n); + }); - let (_, buffer) = self.null_bits(); - let array = Arc::new(av.into_primitive_array::(buffer)) as ArrayRef; + let (_, validity) = self.null_bits(); + let array = Arc::new(to_primitive::(av, validity.clone())) as ArrayRef; array.into() } @@ -142,7 +151,7 @@ where T: DFNumericType let ca: NoNull<_> = self.into_no_null_iter().enumerate().map(f).collect(); ca.into_inner() } else { - self.downcast_iter() + self.into_iter() .enumerate() .map(|(idx, opt_v)| opt_v.map(|v| f((idx, v)))) .collect() @@ -151,7 +160,7 @@ where T: DFNumericType fn apply_with_idx_on_opt(&'a self, f: F) -> Self where F: Fn((usize, Option)) -> Option + Copy { - self.downcast_iter().enumerate().map(f).collect() + self.into_iter().enumerate().map(f).collect() } } @@ -165,8 +174,9 @@ impl<'a> ArrayApply<'a, bool, bool> for DFBooleanArray { let av: AlignedVec<_> = (0..array.len()) .map(|idx| unsafe { f(array.value_unchecked(idx)) }) .collect(); - let null_bit_buffer = array.data_ref().null_buffer().cloned(); - Arc::new(av.into_primitive_array::(null_bit_buffer)) as ArrayRef + + let (_, validity) = self.null_bits(); + Arc::new(to_primitive::(av, validity.clone())) as ArrayRef }) } @@ -177,7 +187,7 @@ impl<'a> ArrayApply<'a, bool, bool> for DFBooleanArray { { self.apply_kernel_cast(|array| { let av: AlignedVec<_> = array.into_iter().map(f).collect(); - Arc::new(av.into_primitive_array::(None)) as ArrayRef + Arc::new(to_primitive::(av, None)) as ArrayRef }) } @@ -207,9 +217,8 @@ impl<'a> ArrayApply<'a, &'a str, Cow<'a, str>> for DFUtf8Array { .map(|idx| unsafe { f(arr.value_unchecked(idx)) }) .collect(); - let null_bit_buffer = self.array.data_ref().null_buffer().cloned(); - let array = Arc::new(av.into_primitive_array::(null_bit_buffer)) as ArrayRef; - + let (_, validity) = self.null_bits(); + let array = Arc::new(to_primitive::(av, validity.clone())) as ArrayRef; array.into() } @@ -219,9 +228,8 @@ impl<'a> ArrayApply<'a, &'a str, Cow<'a, str>> for DFUtf8Array { S: DFNumericType, { let av: AlignedVec<_> = self.downcast_iter().map(f).collect(); - let null_bit_buffer = self.array.data_ref().null_buffer().cloned(); - let array = Arc::new(av.into_primitive_array::(null_bit_buffer)) as ArrayRef; - + let (_, validity) = self.null_bits(); + let array = Arc::new(to_primitive::(av, validity.clone())) as ArrayRef; array.into() } @@ -260,16 +268,16 @@ impl ArrayApplyKernel for DFBooleanArray { } } -impl ArrayApplyKernel> for DataArray +impl ArrayApplyKernel> for DataArray where T: DFNumericType { fn apply_kernel(&self, f: F) -> Self - where F: Fn(&PrimitiveArray) -> ArrayRef { + where F: Fn(&PrimitiveArray) -> ArrayRef { self.apply_kernel_cast(f) } fn apply_kernel_cast(&self, f: F) -> DataArray where - F: Fn(&PrimitiveArray) -> ArrayRef, + F: Fn(&PrimitiveArray) -> ArrayRef, S: DFDataType, { let array = self.downcast_ref(); @@ -278,15 +286,15 @@ where T: DFNumericType } } -impl ArrayApplyKernel for DFUtf8Array { +impl ArrayApplyKernel for DFUtf8Array { fn apply_kernel(&self, f: F) -> Self - where F: Fn(&StringArray) -> ArrayRef { + where F: Fn(&LargeUtf8Array) -> ArrayRef { self.apply_kernel_cast(f) } fn apply_kernel_cast(&self, f: F) -> DataArray where - F: Fn(&StringArray) -> ArrayRef, + F: Fn(&LargeUtf8Array) -> ArrayRef, S: DFDataType, { let array = self.downcast_ref(); diff --git a/common/datavalues/src/arrays/ops/apply_test.rs b/common/datavalues/src/arrays/ops/apply_test.rs index cb8d3441f7f7b..1ecf77312b2a0 100644 --- a/common/datavalues/src/arrays/ops/apply_test.rs +++ b/common/datavalues/src/arrays/ops/apply_test.rs @@ -5,7 +5,9 @@ use std::borrow::Cow; use common_arrow::arrow::array::Array; -use common_arrow::arrow::compute; +use common_arrow::arrow::compute::arithmetics::basic::add; +use common_arrow::arrow::compute::boolean::and; +use common_arrow::arrow::compute::boolean::or; use common_exception::Result; use crate::arrays::ops::apply::ArrayApply; @@ -17,7 +19,7 @@ use crate::DFUtf8Array; use crate::UInt16Type; fn new_test_uint16_array(cap: usize, begin: i32, end: i32) -> DataArray { - let mut builder = PrimitiveArrayBuilder::::new(cap); + let mut builder = PrimitiveArrayBuilder::::with_capacity(cap); (begin..end).for_each(|index| { if index % 3 == 0 { @@ -30,7 +32,7 @@ fn new_test_uint16_array(cap: usize, begin: i32, end: i32) -> DataArray DataArray { - let mut builder = BooleanArrayBuilder::new(cap); + let mut builder = BooleanArrayBuilder::with_capacity(cap); (begin..end).for_each(|index| { if index % 3 == 0 { @@ -43,7 +45,7 @@ fn new_test_boolean_array(cap: usize, begin: i32, end: i32) -> DataArray DFUtf8Array { - let mut builder = Utf8ArrayBuilder::new(cap, 2); + let mut builder = Utf8ArrayBuilder::with_capacity(cap); let s = vec!["ax", "by", "cz", "dm", "13"]; (begin..end).for_each(|index| { @@ -102,7 +104,6 @@ fn test_array_apply() -> Result<()> { assert_eq!(22, values[2].value(2)); assert_eq!(0, values[2].value(3)); assert_eq!(24, values[2].value(4)); - assert_eq!(2, values[3].null_count()); assert_eq!(true, values[3].is_null(0)); assert_eq!(31, values[3].value(1)); @@ -128,8 +129,8 @@ fn test_array_apply_kernel() -> Result<()> { let array2 = new_test_uint16_array(5, 5, 10); let arrays = vec![ - array1.apply_kernel(|arr| Arc::new(compute::add(arr, array2.as_ref()).unwrap())), - array1.apply_kernel_cast(|arr| Arc::new(compute::add(arr, array2.as_ref()).unwrap())), + array1.apply_kernel(|arr| Arc::new(add::add(arr, array2.as_ref()).unwrap())), + array1.apply_kernel_cast(|arr| Arc::new(add::add(arr, array2.as_ref()).unwrap())), ]; let values = vec![arrays[0].downcast_ref(), arrays[1].downcast_ref()]; @@ -203,8 +204,8 @@ fn test_boolean_array_apply_kernel() -> Result<()> { let array2 = new_test_boolean_array(5, 5, 10); let arrays = vec![ - array1.apply_kernel(|arr| Arc::new(compute::and(arr, array2.as_ref()).unwrap())), - array1.apply_kernel_cast(|arr| Arc::new(compute::or(arr, array2.as_ref()).unwrap())), + array1.apply_kernel(|arr| Arc::new(and(arr, array2.as_ref()).unwrap())), + array1.apply_kernel_cast(|arr| Arc::new(or(arr, array2.as_ref()).unwrap())), ]; let values = vec![arrays[0].downcast_ref(), arrays[1].downcast_ref()]; diff --git a/common/datavalues/src/arrays/ops/boolean.rs b/common/datavalues/src/arrays/ops/boolean.rs index 85701ca2bda54..8493673a02242 100644 --- a/common/datavalues/src/arrays/ops/boolean.rs +++ b/common/datavalues/src/arrays/ops/boolean.rs @@ -14,7 +14,7 @@ impl DFBooleanArray { match (self.len(), rhs.len()) { // We use Kleene logic because MySQL uses Kleene logic. (left, right) if left == right => { - let result = compute::and_kleene(self.downcast_ref(), rhs.downcast_ref())?; + let result = compute::boolean_kleene::and(self.downcast_ref(), rhs.downcast_ref())?; Ok(DFBooleanArray::from_arrow_array(result)) } (_, 1) => { @@ -53,7 +53,7 @@ impl DFBooleanArray { match (self.len(), rhs.len()) { // We use Kleene logic because MySQL uses Kleene logic. (left, right) if left == right => { - let result = compute::or_kleene(self.downcast_ref(), rhs.downcast_ref())?; + let result = compute::boolean_kleene::or(self.downcast_ref(), rhs.downcast_ref())?; Ok(DFBooleanArray::from_arrow_array(result)) } (_, 1) => { @@ -89,7 +89,7 @@ impl DFBooleanArray { } pub fn not(&self) -> Result { - let result = compute::not(self.downcast_ref())?; + let result = compute::boolean::not(self.downcast_ref()); Ok(DFBooleanArray::from_arrow_array(result)) } diff --git a/common/datavalues/src/arrays/ops/cast.rs b/common/datavalues/src/arrays/ops/cast.rs index 66d78786cfd87..196b70723421d 100644 --- a/common/datavalues/src/arrays/ops/cast.rs +++ b/common/datavalues/src/arrays/ops/cast.rs @@ -2,6 +2,9 @@ // // SPDX-License-Identifier: Apache-2.0. +use std::sync::Arc; + +use common_arrow::arrow::array::ArrayRef; use common_arrow::arrow::compute::cast; use common_exception::ErrorCode; use common_exception::Result; @@ -37,7 +40,7 @@ where }; } - let ca = cast(&ca.array, &N::data_type().to_arrow())?; + let ca: ArrayRef = Arc::from(cast::cast(ca.array.as_ref(), &N::data_type().to_arrow())?); Ok(ca.into()) } diff --git a/common/datavalues/src/arrays/ops/downcast.rs b/common/datavalues/src/arrays/ops/downcast.rs index 014b23cea9340..36c3c46fd4052 100644 --- a/common/datavalues/src/arrays/ops/downcast.rs +++ b/common/datavalues/src/arrays/ops/downcast.rs @@ -4,17 +4,12 @@ use std::sync::Arc; -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BinaryArray; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::ListArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; -use common_arrow::arrow::array::StructArray; +use common_arrow::arrow::array::*; +use common_arrow::arrow::bitmap::utils::BitmapIter; +use common_arrow::arrow::bitmap::utils::ZipValidity; use crate::arrays::DataArray; -use crate::series::IntoSeries; +use crate::prelude::*; use crate::series::Series; use crate::DFBinaryArray; use crate::DFBooleanArray; @@ -23,10 +18,10 @@ use crate::DFPrimitiveType; use crate::DFStructArray; use crate::DFUtf8Array; -impl AsRef> for DataArray +impl AsRef> for DataArray where T: DFPrimitiveType { - fn as_ref(&self) -> &PrimitiveArray { + fn as_ref(&self) -> &PrimitiveArray { self.downcast_ref() } } @@ -34,22 +29,23 @@ where T: DFPrimitiveType impl DataArray where T: DFPrimitiveType { - pub fn downcast_ref(&self) -> &PrimitiveArray { + pub fn downcast_ref(&self) -> &PrimitiveArray { let arr = &*self.array; - unsafe { &*(arr as *const dyn Array as *const PrimitiveArray) } + unsafe { &*(arr as *const dyn Array as *const PrimitiveArray) } } - pub fn downcast_iter(&self) -> impl Iterator> + DoubleEndedIterator { + pub fn downcast_iter(&self) -> ZipValidity<'_, &'_ T::Native, std::slice::Iter<'_, T::Native>> { let arr = &*self.array; - let arr = unsafe { &*(arr as *const dyn Array as *const PrimitiveArray) }; + let arr = unsafe { &*(arr as *const dyn Array as *const PrimitiveArray) }; arr.iter() } pub fn collect_values(&self) -> Vec> { - self.downcast_iter().collect() + let e = self.downcast_iter().map(|c| c.copied()); + e.collect() } - pub fn from_arrow_array(array: PrimitiveArray) -> Self { + pub fn from_arrow_array(array: PrimitiveArray) -> Self { let array_ref = Arc::new(array) as ArrayRef; array_ref.into() } @@ -67,7 +63,7 @@ impl DFBooleanArray { unsafe { &*(arr as *const dyn Array as *const BooleanArray) } } - pub fn downcast_iter(&self) -> impl Iterator> + DoubleEndedIterator { + pub fn downcast_iter(&self) -> ZipValidity<'_, bool, BitmapIter<'_>> { let arr = &*self.array; let arr = unsafe { &*(arr as *const dyn Array as *const BooleanArray) }; arr.iter() @@ -83,72 +79,82 @@ impl DFBooleanArray { } } -impl AsRef for DFUtf8Array { - fn as_ref(&self) -> &StringArray { +impl AsRef for DFUtf8Array { + fn as_ref(&self) -> &LargeUtf8Array { self.downcast_ref() } } impl DFUtf8Array { - pub fn downcast_ref(&self) -> &StringArray { + pub fn downcast_ref(&self) -> &LargeUtf8Array { let arr = &*self.array; - unsafe { &*(arr as *const dyn Array as *const StringArray) } + unsafe { &*(arr as *const dyn Array as *const LargeUtf8Array) } } - pub fn downcast_iter<'a>(&self) -> impl Iterator> + DoubleEndedIterator { + pub fn downcast_iter(&self) -> ZipValidity<'_, &'_ str, Utf8ValuesIter<'_, i64>> { let arr = &*self.array; - let arr = unsafe { &*(arr as *const dyn Array as *const StringArray) }; + let arr = unsafe { &*(arr as *const dyn Array as *const LargeUtf8Array) }; arr.iter() } - pub fn collect_values<'a>(&self) -> Vec> { + pub fn collect_values(&self) -> Vec> { self.downcast_iter().collect() } - pub fn from_arrow_array(array: StringArray) -> Self { + pub fn from_arrow_array(array: LargeUtf8Array) -> Self { let array_ref = Arc::new(array) as ArrayRef; array_ref.into() } } -impl AsRef for DFListArray { - fn as_ref(&self) -> &ListArray { +impl AsRef for DFListArray { + fn as_ref(&self) -> &LargeListArray { self.downcast_ref() } } impl DFListArray { - pub fn downcast_ref(&self) -> &ListArray { + pub fn downcast_ref(&self) -> &LargeListArray { let arr = &*self.array; - unsafe { &*(arr as *const dyn Array as *const ListArray) } + unsafe { &*(arr as *const dyn Array as *const LargeListArray) } } pub fn downcast_iter(&self) -> impl Iterator> + DoubleEndedIterator { let arr = &*self.array; - let arr = unsafe { &*(arr as *const dyn Array as *const ListArray) }; + let arr = unsafe { &*(arr as *const dyn Array as *const LargeListArray) }; - arr.iter().map(|a| a.map(|a| a.into_series())) + arr.iter().map(|a| { + a.map(|b| { + let c: ArrayRef = Arc::from(b); + c.into_series() + }) + }) } - pub fn from_arrow_array(array: ListArray) -> Self { + pub fn from_arrow_array(array: LargeListArray) -> Self { let array_ref = Arc::new(array) as ArrayRef; array_ref.into() } } -impl AsRef for DFBinaryArray { - fn as_ref(&self) -> &BinaryArray { +impl AsRef for DFBinaryArray { + fn as_ref(&self) -> &LargeBinaryArray { self.downcast_ref() } } impl DFBinaryArray { - pub fn downcast_ref(&self) -> &BinaryArray { + pub fn downcast_ref(&self) -> &LargeBinaryArray { let arr = &*self.array; - unsafe { &*(arr as *const dyn Array as *const BinaryArray) } + unsafe { &*(arr as *const dyn Array as *const LargeBinaryArray) } + } + + pub fn collect_values(&self) -> Vec>> { + let e = self.downcast_ref().iter().map(|c| c.map(|d| d.to_owned())); + e.collect() } - pub fn from_arrow_array(array: BinaryArray) -> Self { + pub fn from_arrow_array(array: LargeBinaryArray) -> Self { let array_ref = Arc::new(array) as ArrayRef; array_ref.into() } diff --git a/common/datavalues/src/arrays/ops/downcast_test.rs b/common/datavalues/src/arrays/ops/downcast_test.rs index 5464dbcb0f49a..02fe2e2a9afd1 100644 --- a/common/datavalues/src/arrays/ops/downcast_test.rs +++ b/common/datavalues/src/arrays/ops/downcast_test.rs @@ -2,79 +2,59 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::array::BinaryArray; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::Int32Array; -use common_arrow::arrow::array::ListArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; -use common_arrow::arrow::array::StructArray; -use common_arrow::arrow::datatypes::DataType; -use common_arrow::arrow::datatypes::Field; -use common_arrow::arrow::datatypes::Schema; -use common_arrow::arrow::record_batch::RecordBatch; +use common_arrow::arrow::array::*; use common_exception::Result; use crate::arrays::builders::*; use crate::prelude::*; -use crate::DFBinaryArray; use crate::DFBooleanArray; -use crate::DFListArray; -use crate::DFStructArray; use crate::DFUInt16Array; use crate::DFUtf8Array; -use crate::Int32Type; use crate::UInt16Type; #[test] fn test_array_as_ref() -> Result<()> { // Test DFUint16Array let df_uint16_array = DFUInt16Array::new_from_iter(1u16..4u16); - let arrow_uint16_array: &PrimitiveArray = df_uint16_array.as_ref(); - assert_eq!(&[1u16, 2, 3], &arrow_uint16_array.values()); + let arrow_uint16_array = df_uint16_array.as_ref(); + assert_eq!(&[1u16, 2, 3], &arrow_uint16_array.values().as_slice()); // Test DFBooleanArray - let mut boolean_builder = BooleanArrayBuilder::new(3); + let mut boolean_builder = BooleanArrayBuilder::with_capacity(3); boolean_builder.append_value(true); boolean_builder.append_value(false); boolean_builder.append_value(true); let df_boolean_array = boolean_builder.finish(); let arrow_boolean_array: &BooleanArray = df_boolean_array.as_ref(); // 5 means 0b_101 - assert_eq!(&[5], &arrow_boolean_array.values().as_slice()); + assert_eq!(1, arrow_boolean_array.values().null_count()); + assert_eq!(false, arrow_boolean_array.values().get_bit(1)); // Test DFUtf8Array - let mut utf8_builder = Utf8ArrayBuilder::new(3, 2); + let mut utf8_builder = Utf8ArrayBuilder::with_capacity(3); utf8_builder.append_value("1a"); utf8_builder.append_value("2b"); utf8_builder.append_value("3c"); let df_utf8_array = utf8_builder.finish(); let arrow_string_array = df_utf8_array.as_ref(); - assert_eq!( - &"1a2b3c".as_bytes(), - &arrow_string_array.value_data().as_slice() - ); + assert_eq!(b"1a2b3c", arrow_string_array.values().as_slice()); // Test DFListArray - let mut list_value_builder = ArrowPrimitiveArrayBuilder::::new(3); - list_value_builder.append_slice(&[1u16, 2u16, 3u16]); - let mut list_builder = ListPrimitiveArrayBuilder::::new(list_value_builder, 3); + let mut list_builder = ListPrimitiveArrayBuilder::::with_capacity(3, 0); + list_builder.append_slice(Some(&[1u16, 2u16, 3u16])); let df_list = list_builder.finish(); let arrow_list = df_list.as_ref(); - let expected = "PrimitiveArray\n[\n 1,\n 2,\n 3,\n]"; - assert_eq!(expected, format!("{:?}", arrow_list.values())); + let first_array: ArrayRef = Arc::from(arrow_list.value(0)); + let first_array = DFUInt16Array::from(first_array); + let vs: Vec<_> = first_array.into_no_null_iter().collect(); + assert_eq!(vs, vec![1u16, 2u16, 3u16]); // Test DFBinaryArray - let mut binary_builder = BinaryArrayBuilder::new(8); + let mut binary_builder = BinaryArrayBuilder::with_capacity(8); binary_builder.append_value(&"123"); let df_binary_array = binary_builder.finish(); - let arrow_binary_array = df_binary_array.as_ref(); - assert_eq!( - &[0x31, 0x32, 0x33], - &arrow_binary_array.value_data().as_slice() - ); - - // TODO: Test DFStructArray + let array_ref = df_binary_array.downcast_ref(); + assert_eq!(b"123", array_ref.value(0)); Ok(()) } @@ -84,69 +64,24 @@ fn test_array_as_ref() -> Result<()> { fn test_array_downcast() -> Result<()> { // Test PrimitiveArray let vec_uint16 = vec![1u16, 2u16, 3u16]; - let arrow_array = PrimitiveArray::::from_iter_values(vec_uint16.into_iter()); + let arrow_array = PrimitiveArray::::from_trusted_len_values_iter(vec_uint16.into_iter()); let df_array = DFUInt16Array::from_arrow_array(arrow_array); let values = df_array.collect_values(); assert_eq!(&[Some(1u16), Some(2u16), Some(3u16)], values.as_slice()); // Test BooleanArray let vec_bool = vec![true, false, true]; - let arrow_bool_array = BooleanArray::from(vec_bool); - let df_bool_array = DFBooleanArray::from_arrow_array(arrow_bool_array); - let values = df_bool_array.collect_values(); + let arrow_bool_array = BooleanArray::from_slice(&vec_bool); + let df_array = DFBooleanArray::from_arrow_array(arrow_bool_array); + let values = df_array.collect_values(); assert_eq!(&[Some(true), Some(false), Some(true)], values.as_slice()); // Test Utf8Array let vec_str = vec![Some("foo"), None, Some("bar")]; - let arrow_str_array = StringArray::from(vec_str); - let df_bool_array = DFUtf8Array::from_arrow_array(arrow_str_array); - let values = df_bool_array.collect_values(); + let arrow_str_array = LargeUtf8Array::from(vec_str); + let df_array = DFUtf8Array::from_arrow_array(arrow_str_array); + let values = df_array.collect_values(); assert_eq!(&[Some("foo"), None, Some("bar")], values.as_slice()); - // Test ListArray - let data = vec![ - Some(vec![Some(0), Some(1), Some(2)]), - Some(Vec::>::new()), - Some(vec![Some(3), None, Some(5)]), - Some(vec![Some(6), Some(7)]), - ]; - let arrow_list_array = ListArray::from_iter_primitive::(data); - let df_list_array = DFListArray::from_arrow_array(arrow_list_array); - let values: Vec> = df_list_array.downcast_iter().collect(); - - let expected = vec![ - Series::new(vec![Some(0), Some(1), Some(2)]), - Series::new(Vec::>::new()), - Series::new(vec![Some(3), None, Some(5)]), - Series::new(vec![Some(6), Some(7)]), - ]; - for i in 0..expected.len() { - assert_eq!( - &values[i].as_ref().unwrap().to_values()?, - &expected[i].to_values()? - ); - } - - // Test BinaryArray - let arrow_binary_array = BinaryArray::from_opt_vec(vec![Some(b"1a"), None, Some(b"2b")]); - let df_binary_array = DFBinaryArray::from_arrow_array(arrow_binary_array); - let downcast_array = df_binary_array.downcast_ref(); - assert_eq!( - &[0x31, 0x61, 0x32, 0x62], - downcast_array.value_data().as_slice() - ); - - // Test StructArray - let id_array = Int32Array::from(vec![1, 2, 3, 4, 5]); - let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); - - let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(id_array)])?; - - let arrow_struct_array = StructArray::from(batch); - let df_struct_array = DFStructArray::from_arrow_array(arrow_struct_array); - let downcast_struct_array = df_struct_array.downcast_ref(); - let expected = "StructArray\n[\n-- child 0: \"id\" (Int32)\nPrimitiveArray\n[\n 1,\n 2,\n 3,\n 4,\n 5,\n]\n]"; - assert_eq!(expected, format!("{:?}", downcast_struct_array)); - Ok(()) } diff --git a/common/datavalues/src/arrays/ops/fill.rs b/common/datavalues/src/arrays/ops/fill.rs index f27c62589614a..08376fb31b621 100644 --- a/common/datavalues/src/arrays/ops/fill.rs +++ b/common/datavalues/src/arrays/ops/fill.rs @@ -5,9 +5,11 @@ use common_exception::Result; use crate::arrays::builders::Utf8ArrayBuilder; +use crate::arrays::BinaryArrayBuilder; use crate::arrays::DataArray; use crate::series::Series; use crate::utils::NoNull; +use crate::DFBinaryArray; use crate::DFBooleanArray; use crate::DFListArray; use crate::DFPrimitiveType; @@ -98,7 +100,7 @@ impl ArrayFullNull for DFBooleanArray { impl<'a> ArrayFull<&'a str> for DFUtf8Array { fn full(value: &'a str, length: usize) -> Self { - let mut builder = Utf8ArrayBuilder::new(length, length * value.len()); + let mut builder = Utf8ArrayBuilder::with_capacity(length * value.len()); for _ in 0..length { builder.append_value(value); @@ -126,3 +128,23 @@ impl ArrayFullNull for DFListArray { todo!() } } + +impl ArrayFull<&[u8]> for DFBinaryArray { + fn full(value: &[u8], length: usize) -> DFBinaryArray { + let mut builder = BinaryArrayBuilder::with_capacity(length); + for _ in 0..length { + builder.append_value(value); + } + builder.finish() + } +} + +impl ArrayFullNull for DFBinaryArray { + fn full_null(length: usize) -> DFBinaryArray { + let mut builder = BinaryArrayBuilder::with_capacity(length); + for _ in 0..length { + builder.append_null(); + } + builder.finish() + } +} diff --git a/common/datavalues/src/arrays/ops/fill_test.rs b/common/datavalues/src/arrays/ops/fill_test.rs index ef85c1918e70b..4dac9edecb0fb 100644 --- a/common/datavalues/src/arrays/ops/fill_test.rs +++ b/common/datavalues/src/arrays/ops/fill_test.rs @@ -2,8 +2,6 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; use common_exception::Result; use crate::arrays::ops::fill::ArrayFull; @@ -11,14 +9,13 @@ use crate::arrays::ops::fill::ArrayFullNull; use crate::DFBooleanArray; use crate::DFUInt16Array; use crate::DFUtf8Array; -use crate::UInt16Type; #[test] fn test_array_fill() -> Result<()> { // Test full for PrimitiveArray let mut df_uint16_array = DFUInt16Array::full(5u16, 3); - let arrow_uint16_array: &PrimitiveArray = df_uint16_array.as_ref(); - assert_eq!(&[5u16, 5u16, 5u16], &arrow_uint16_array.values()); + let arrow_uint16_array = df_uint16_array.as_ref(); + assert_eq!(&[5u16, 5u16, 5u16], &arrow_uint16_array.values().as_slice()); // Test full_null for PrimitiveArray df_uint16_array = DFUInt16Array::full_null(3); assert_eq!(3, df_uint16_array.null_count()); @@ -28,9 +25,8 @@ fn test_array_fill() -> Result<()> { // Test full for BooleanArray let mut df_boolean_array = DFBooleanArray::full(true, 3); - let arrow_boolean_array: &BooleanArray = df_boolean_array.as_ref(); - // 7 means 0b_111 - assert_eq!(&[7], &arrow_boolean_array.values().as_slice()); + assert_eq!(0, df_boolean_array.null_count()); + // Test full_null for BooleanArray df_boolean_array = DFBooleanArray::full_null(3); assert_eq!(3, df_boolean_array.null_count()); diff --git a/common/datavalues/src/arrays/ops/group_hash.rs b/common/datavalues/src/arrays/ops/group_hash.rs index 103353c46c1ff..b9695e388b45e 100644 --- a/common/datavalues/src/arrays/ops/group_hash.rs +++ b/common/datavalues/src/arrays/ops/group_hash.rs @@ -26,7 +26,7 @@ impl GroupHash for DFUInt8Array { fn group_hash(&self, ptr: usize, step: usize) -> Result<()> { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const u8, ptr as *mut u8, 1); } @@ -42,7 +42,7 @@ impl GroupHash for DFInt8Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const i8 as *const u8, ptr as *mut u8, 1); } @@ -58,7 +58,7 @@ impl GroupHash for DFUInt16Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const u16 as *const u8, ptr as *mut u8, 2); } @@ -74,7 +74,7 @@ impl GroupHash for DFInt16Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const i16 as *const u8, ptr as *mut u8, 2); } @@ -90,7 +90,7 @@ impl GroupHash for DFInt32Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const i32 as *const u8, ptr as *mut u8, 4); } @@ -106,7 +106,7 @@ impl GroupHash for DFUInt32Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const u32 as *const u8, ptr as *mut u8, 4); } @@ -139,7 +139,7 @@ impl GroupHash for DFFloat32Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { let bits = value.to_bits(); std::ptr::copy_nonoverlapping(&bits as *const u32 as *const u8, ptr as *mut u8, 4); @@ -155,7 +155,7 @@ impl GroupHash for DFFloat64Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { let bits = value.to_bits(); std::ptr::copy_nonoverlapping(&bits as *const u64 as *mut u8, ptr as *mut u8, 8); @@ -172,7 +172,7 @@ impl GroupHash for DFUInt64Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const u64 as *mut u8, ptr as *mut u8, 8); } @@ -187,7 +187,7 @@ impl GroupHash for DFInt64Array { let mut ptr = ptr; let array = self.downcast_ref(); - for value in array.values() { + for value in array.values().as_slice() { unsafe { std::ptr::copy_nonoverlapping(value as *const i64 as *mut u8, ptr as *mut u8, 8); } diff --git a/common/datavalues/src/arrays/ops/scatter.rs b/common/datavalues/src/arrays/ops/scatter.rs index 6989cc25a8663..5eab2da879d87 100644 --- a/common/datavalues/src/arrays/ops/scatter.rs +++ b/common/datavalues/src/arrays/ops/scatter.rs @@ -60,7 +60,7 @@ where T: DFNumericType let mut builders = Vec::with_capacity(scattered_size); for _i in 0..scattered_size { - builders.push(PrimitiveArrayBuilder::::new(self.len())); + builders.push(PrimitiveArrayBuilder::::with_capacity(self.len())); } match self.null_count() { @@ -100,10 +100,7 @@ impl ArrayScatter for DFUtf8Array { let mut builders = Vec::with_capacity(scattered_size); for _i in 0..scattered_size { - builders.push(Utf8ArrayBuilder::new( - self.len(), - self.get_array_memory_size(), - )); + builders.push(Utf8ArrayBuilder::with_capacity(self.len())); } match self.null_count() { @@ -143,7 +140,7 @@ impl ArrayScatter for DFBooleanArray { let mut builders = Vec::with_capacity(scattered_size); for _i in 0..scattered_size { - builders.push(BooleanArrayBuilder::new(self.len())); + builders.push(BooleanArrayBuilder::with_capacity(self.len())); } match self.null_count() { @@ -226,7 +223,7 @@ impl ArrayScatter for DFBinaryArray { let mut builders = Vec::with_capacity(scattered_size); let guess_scattered_len = ((self.len() as f64) * 1.1 / (scattered_size as f64)) as usize; for _i in 0..scattered_size { - let builder = BinaryArrayBuilder::new(guess_scattered_len); + let builder = BinaryArrayBuilder::with_capacity(guess_scattered_len); builders.push(builder); } diff --git a/common/datavalues/src/arrays/ops/scatter_test.rs b/common/datavalues/src/arrays/ops/scatter_test.rs index 1a863c50dab1a..8115b3c5a8b48 100644 --- a/common/datavalues/src/arrays/ops/scatter_test.rs +++ b/common/datavalues/src/arrays/ops/scatter_test.rs @@ -2,6 +2,7 @@ // // SPDX-License-Identifier: Apache-2.0. +use common_arrow::arrow::array::ArrayRef; use common_exception::Result; use crate::arrays::builders::*; @@ -22,10 +23,10 @@ fn test_scatter() -> Result<()> { assert_eq!(df_uint16_array.len(), indices.len()); let array_vec = unsafe { df_uint16_array.scatter_unchecked(&mut indices.into_iter(), 4)? }; - assert_eq!(&[7u16, 10], &array_vec[0].as_ref().values()); - assert_eq!(&[1u16, 4, 9], &array_vec[1].as_ref().values()); - assert_eq!(&[2u16, 6], &array_vec[2].as_ref().values()); - assert_eq!(&[3u16, 5, 8], &array_vec[3].as_ref().values()); + assert_eq!(&[7u16, 10], array_vec[0].as_ref().values().as_slice()); + assert_eq!(&[1u16, 4, 9], array_vec[1].as_ref().values().as_slice()); + assert_eq!(&[2u16, 6], array_vec[2].as_ref().values().as_slice()); + assert_eq!(&[3u16, 5, 8], array_vec[3].as_ref().values().as_slice()); // Test DFUint16Array let df_utf8_array = DFUtf8Array::new_from_slice(&["a", "b", "c", "d"]); @@ -33,14 +34,10 @@ fn test_scatter() -> Result<()> { assert_eq!(df_utf8_array.len(), indices.len()); let array_vec = unsafe { df_utf8_array.scatter_unchecked(&mut indices.into_iter(), 2)? }; - assert_eq!( - &"b".as_bytes(), - &array_vec[0].as_ref().value_data().as_slice() - ); - assert_eq!( - &"acd".as_bytes(), - &array_vec[1].as_ref().value_data().as_slice() - ); + let v1: Vec<&str> = array_vec[0].into_no_null_iter().collect(); + let v2: Vec<&str> = array_vec[1].into_no_null_iter().collect(); + assert_eq!(vec!["b"], v1); + assert_eq!(vec!["a", "c", "d"], v2); // Test BooleanArray let df_bool_array = DFBooleanArray::new_from_slice(&[true, false, true, false]); @@ -48,28 +45,37 @@ fn test_scatter() -> Result<()> { assert_eq!(df_bool_array.len(), indices.len()); let array_vec = unsafe { df_bool_array.scatter_unchecked(&mut indices.into_iter(), 2)? }; - assert_eq!(&[2], &array_vec[0].as_ref().values().as_slice()); - assert_eq!(&[1], &array_vec[1].as_ref().values().as_slice()); + assert_eq!( + vec![false, true], + array_vec[0].into_no_null_iter().collect::>() + ); + assert_eq!( + vec![true, false], + array_vec[1].into_no_null_iter().collect::>() + ); // Test BinaryArray - let mut binary_builder = BinaryArrayBuilder::new(8); + let mut binary_builder = BinaryArrayBuilder::with_capacity(8); binary_builder.append_value(&"12"); binary_builder.append_value(&"ab"); - binary_builder.append_value(&"c"); - binary_builder.append_value(&"3"); + binary_builder.append_value(&"c1"); + binary_builder.append_value(&"32"); let df_binary_array = binary_builder.finish(); let indices = vec![1, 0, 0, 1]; let array_vec = unsafe { df_binary_array.scatter_unchecked(&mut indices.into_iter(), 2)? }; - assert_eq!( - [b'a', b'b', b'c'], - array_vec[0].as_ref().value_data().as_slice() - ); - assert_eq!( - [b'1', b'2', b'3'], - array_vec[1].as_ref().value_data().as_slice() - ); - // Test ListArray + let values: Vec> = (0..array_vec[0].len()) + .map(|idx| array_vec[0].downcast_ref().value(idx).to_vec()) + .collect(); + + assert_eq!(vec![b"ab".to_vec(), b"c1".to_vec()], values); + + let values: Vec> = (0..array_vec[1].len()) + .map(|idx| array_vec[1].downcast_ref().value(idx).to_vec()) + .collect(); + assert_eq!(vec![b"12".to_vec(), b"32".to_vec()], values); + + // Test LargeListArray let mut builder = get_list_builder(&DataType::UInt16, 12, 3); builder.append_series(&Series::new(vec![1_u16, 2, 3])); builder.append_series(&Series::new(vec![7_u16, 8, 9])); @@ -80,10 +86,21 @@ fn test_scatter() -> Result<()> { let indices = vec![1, 0, 0, 1]; let array_vec = unsafe { df_list.scatter_unchecked(&mut indices.into_iter(), 2)? }; - let expected1 = "PrimitiveArray\n[\n 7,\n 8,\n 9,\n 10,\n 11,\n 12,\n]"; - let expected2 = "PrimitiveArray\n[\n 1,\n 2,\n 3,\n 4,\n 5,\n 6,\n]"; - assert_eq!(expected1, format!("{:?}", array_vec[0].as_ref().values())); - assert_eq!(expected2, format!("{:?}", array_vec[1].as_ref().values())); + let c0: ArrayRef = Arc::from(array_vec[0].downcast_ref().value(0)); + let c0 = DFUInt16Array::from(c0); + let c1: ArrayRef = Arc::from(array_vec[1].downcast_ref().value(0)); + let c1 = DFUInt16Array::from(c1); + + assert_eq!(&[7, 8, 9], c0.downcast_ref().values().as_slice()); + assert_eq!(&[1, 2, 3], c1.downcast_ref().values().as_slice()); + + let c0: ArrayRef = Arc::from(array_vec[0].downcast_ref().value(1)); + let c0 = DFUInt16Array::from(c0); + let c1: ArrayRef = Arc::from(array_vec[1].downcast_ref().value(1)); + let c1 = DFUInt16Array::from(c1); + + assert_eq!(&[10, 11, 12], c0.downcast_ref().values().as_slice()); + assert_eq!(&[4, 5, 6], c1.downcast_ref().values().as_slice()); Ok(()) } diff --git a/common/datavalues/src/arrays/ops/take.rs b/common/datavalues/src/arrays/ops/take.rs index 40a2fa27f9bf8..40da07461c3a2 100644 --- a/common/datavalues/src/arrays/ops/take.rs +++ b/common/datavalues/src/arrays/ops/take.rs @@ -10,8 +10,8 @@ use std::fmt::Debug; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::compute; +use common_arrow::arrow::array::*; +use common_arrow::arrow::compute::take; use common_exception::ErrorCode; use common_exception::Result; @@ -19,9 +19,9 @@ use super::TakeIdx; use crate::arrays::kernels::*; use crate::arrays::DataArray; use crate::prelude::*; -use crate::utils::NoNull; use crate::*; +// TODO add unchecked take pub trait ArrayTake: Debug { /// Take values from DataArray by index. /// @@ -54,14 +54,6 @@ pub trait ArrayTake: Debug { } } -/// Traverse and collect every nth element -pub trait ArrayTakeEvery: Debug { - /// Traverse and collect every nth element in a new array. - fn take_every(&self, _n: usize) -> DataArray { - unimplemented!() - } -} - macro_rules! take_iter_n_arrays { ($ca:expr, $indices:expr) => {{ let taker = $ca.take_rand(); @@ -95,38 +87,23 @@ where T: DFNumericType if self.is_empty() { return Ok(Self::full_null(array.len())); } - - // take(chunks.next(), array, None), - match self.null_count() { - 0 => Ok(Self::from( - take_no_null_primitive(primitive_array, array) as ArrayRef - )), - _ => { - let taked_array = compute::take(self.array.as_ref(), array, None)?; - Ok(Self::from(taked_array)) - } - } + let taked_array = take::take(self.array.as_ref(), array)?; + Ok(Self::from(taked_array)) } TakeIdx::Iter(iter) => { if self.is_empty() { return Ok(Self::full_null(iter.size_hint().0)); } - let array = match self.null_count() { - 0 => take_no_null_primitive_iter_unchecked(primitive_array, iter) as ArrayRef, - _ => take_primitive_iter_unchecked(primitive_array, iter) as ArrayRef, - }; + let array = + take_primitive_iter_unchecked::(primitive_array, iter) as ArrayRef; Ok(Self::from(array)) } TakeIdx::IterNulls(iter) => { if self.is_empty() { return Ok(Self::full_null(iter.size_hint().0)); } - let array = match self.null_count() { - 0 => { - take_no_null_primitive_opt_iter_unchecked(primitive_array, iter) as ArrayRef - } - _ => take_primitive_opt_iter_unchecked(primitive_array, iter) as ArrayRef, - }; + let array = + take_primitive_opt_iter_unchecked::(primitive_array, iter) as ArrayRef; Ok(Self::from(array)) } } @@ -138,29 +115,7 @@ where T: DFNumericType I: Iterator, INulls: Iterator>, { - let primitive_array = self.downcast_ref(); - match indices { - TakeIdx::Array(array) => { - if self.is_empty() { - return Ok(Self::full_null(array.len())); - } - let array = compute::take(array, array, None)?; - Ok(Self::from(array)) - } - TakeIdx::Iter(iter) => { - if self.is_empty() { - return Ok(Self::full_null(iter.size_hint().0)); - } - let array = match self.null_count() { - 0 => take_no_null_primitive_iter(primitive_array, iter) as ArrayRef, - _ => take_primitive_iter(primitive_array, iter) as ArrayRef, - }; - Ok(Self::from(array)) - } - TakeIdx::IterNulls(_) => { - panic!("not supported in take, only supported in take_unchecked for the join operation") - } - } + unsafe { self.take_unchecked(indices) } } } @@ -177,27 +132,21 @@ impl ArrayTake for DFBooleanArray { if self.is_empty() { return Ok(Self::full_null(array.len())); } - let array = compute::take(array, array, None)?; + let array = take::take(array, array)?; Ok(Self::from(array)) } TakeIdx::Iter(iter) => { if self.is_empty() { return Ok(Self::full_null(iter.size_hint().0)); } - let array = match self.null_count() { - 0 => take_no_null_bool_iter_unchecked(boolean_array, iter) as ArrayRef, - _ => take_bool_iter_unchecked(boolean_array, iter) as ArrayRef, - }; + let array = take_bool_iter_unchecked(boolean_array, iter) as ArrayRef; Ok(Self::from(array)) } TakeIdx::IterNulls(iter) => { if self.is_empty() { return Ok(Self::full_null(iter.size_hint().0)); } - let array = match self.null_count() { - 0 => take_no_null_bool_opt_iter_unchecked(boolean_array, iter) as ArrayRef, - _ => take_bool_opt_iter_unchecked(boolean_array, iter) as ArrayRef, - }; + let array = take_bool_opt_iter_unchecked(boolean_array, iter) as ArrayRef; Ok(Self::from(array)) } } @@ -209,29 +158,7 @@ impl ArrayTake for DFBooleanArray { I: Iterator, INulls: Iterator>, { - let boolean_array = self.downcast_ref(); - match indices { - TakeIdx::Array(array) => { - if self.is_empty() { - return Ok(Self::full_null(array.len())); - } - let array = compute::take(boolean_array, array, None)?; - Ok(Self::from(array)) - } - TakeIdx::Iter(iter) => { - if self.is_empty() { - return Ok(Self::full_null(iter.size_hint().0)); - } - let array = match self.null_count() { - 0 => take_no_null_bool_iter(boolean_array, iter) as ArrayRef, - _ => take_bool_iter(boolean_array, iter) as ArrayRef, - }; - Ok(Self::from(array)) - } - TakeIdx::IterNulls(_) => { - panic!("not supported in take, only supported in take_unchecked for the join operation") - } - } + unsafe { self.take_unchecked(indices) } } } @@ -245,27 +172,21 @@ impl ArrayTake for DFUtf8Array { let str_array = self.downcast_ref(); match indices { TakeIdx::Array(array) => { - let array = compute::take(str_array, array, None)?; + let array = take::take(str_array, array)?; Ok(Self::from(array)) } TakeIdx::Iter(iter) => { if self.is_empty() { return Ok(Self::full_null(iter.size_hint().0)); } - let array = match self.null_count() { - 0 => take_no_null_utf8_iter_unchecked(str_array, iter) as ArrayRef, - _ => take_utf8_iter_unchecked(str_array, iter) as ArrayRef, - }; + let array = take_utf8_iter_unchecked(str_array, iter) as ArrayRef; Ok(Self::from(array)) } TakeIdx::IterNulls(iter) => { if self.is_empty() { return Ok(Self::full_null(iter.size_hint().0)); } - let array = match self.null_count() { - 0 => take_no_null_utf8_opt_iter_unchecked(str_array, iter) as ArrayRef, - _ => take_utf8_opt_iter_unchecked(str_array, iter) as ArrayRef, - }; + let array = take_utf8_opt_iter_unchecked(str_array, iter) as ArrayRef; Ok(Self::from(array)) } } @@ -277,26 +198,7 @@ impl ArrayTake for DFUtf8Array { I: Iterator, INulls: Iterator>, { - let str_array = self.downcast_ref(); - match indices { - TakeIdx::Array(array) => { - let array = compute::take(str_array, array, None)?; - Ok(Self::from(array)) - } - TakeIdx::Iter(iter) => { - if self.is_empty() { - return Ok(Self::full_null(iter.size_hint().0)); - } - let array = match self.null_count() { - 0 => take_no_null_utf8_iter(str_array, iter) as ArrayRef, - _ => take_utf8_iter(str_array, iter) as ArrayRef, - }; - Ok(Self::from(array)) - } - TakeIdx::IterNulls(_) => { - panic!("not supported in take, only supported in take_unchecked for the join operation") - } - } + unsafe { self.take_unchecked(indices) } } } @@ -319,7 +221,7 @@ impl ArrayTake for DFListArray { let list_array = self.downcast_ref(); match indices { TakeIdx::Array(array) => { - let array = compute::take(list_array, array, None)?; + let array = take::take(list_array, array)?; Ok(Self::from(array)) } TakeIdx::Iter(iter) => { @@ -354,50 +256,3 @@ pub trait AsTakeIndex { fn take_index_len(&self) -> usize; } - -impl ArrayTakeEvery for DataArray -where T: DFNumericType -{ - fn take_every(&self, n: usize) -> DataArray { - if self.null_count() == 0 { - let a: NoNull<_> = self.into_no_null_iter().step_by(n).collect(); - a.into_inner() - } else { - self.downcast_iter().step_by(n).collect() - } - } -} - -impl ArrayTakeEvery for DFBooleanArray { - fn take_every(&self, n: usize) -> DFBooleanArray { - if self.null_count() == 0 { - self.into_no_null_iter().step_by(n).collect() - } else { - self.downcast_iter().step_by(n).collect() - } - } -} - -impl ArrayTakeEvery for DFUtf8Array { - fn take_every(&self, n: usize) -> DFUtf8Array { - if self.null_count() == 0 { - self.into_no_null_iter().step_by(n).collect() - } else { - self.downcast_iter().step_by(n).collect() - } - } -} - -impl ArrayTakeEvery for DFListArray { - fn take_every(&self, n: usize) -> DFListArray { - if self.null_count() == 0 { - self.into_no_null_iter().step_by(n).collect() - } else { - self.downcast_iter().step_by(n).collect() - } - } -} - -impl ArrayTakeEvery for DFNullArray {} -impl ArrayTakeEvery for DFStructArray {} -impl ArrayTakeEvery for DFBinaryArray {} diff --git a/common/datavalues/src/arrays/ops/take_random.rs b/common/datavalues/src/arrays/ops/take_random.rs index 0a690c78aaa6f..f1be5a63a13d7 100644 --- a/common/datavalues/src/arrays/ops/take_random.rs +++ b/common/datavalues/src/arrays/ops/take_random.rs @@ -1,14 +1,13 @@ // Copyright 2020-2021 The Datafuse Authors. // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::ListArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; -use common_arrow::arrow::array::UInt32Array; +use std::sync::Arc; + +use common_arrow::arrow::array::*; use crate::arrays::DataArray; +use crate::prelude::LargeListArray; +use crate::prelude::LargeUtf8Array; use crate::series::IntoSeries; use crate::series::Series; use crate::series::SeriesWrap; @@ -156,7 +155,7 @@ where T: DFNumericType } pub struct Utf8TakeRandom<'a> { - arr: &'a StringArray, + arr: &'a LargeUtf8Array, } impl<'a> TakeRandom for Utf8TakeRandom<'a> { @@ -230,7 +229,7 @@ where T: Copy pub struct NumTakeRandomSingleArray<'a, T> where T: DFNumericType { - arr: &'a PrimitiveArray, + arr: &'a PrimitiveArray, } impl<'a, T> TakeRandom for NumTakeRandomSingleArray<'a, T> @@ -268,7 +267,7 @@ impl<'a> TakeRandom for BoolTakeRandom<'a> { } pub struct ListTakeRandom<'a> { - arr: &'a ListArray, + arr: &'a LargeListArray, } impl<'a> TakeRandom for ListTakeRandom<'a> { @@ -276,13 +275,17 @@ impl<'a> TakeRandom for ListTakeRandom<'a> { #[inline] fn get(&self, index: usize) -> Option { - let v = take_random_get!(self, index); + let v: Option> = take_random_get!(self, index); + let v = v.map(|c| { + let c: Arc = Arc::from(c); + c + }); v.map(|v| v.into_series()) } #[inline] unsafe fn get_unchecked(&self, index: usize) -> Self::Item { - let v = self.arr.value_unchecked(index); + let v: Arc = Arc::from(self.arr.value_unchecked(index)); v.into_series() } } diff --git a/common/datavalues/src/arrays/ops/take_single.rs b/common/datavalues/src/arrays/ops/take_single.rs index ce5fdc3a4c638..343ff197b128c 100644 --- a/common/datavalues/src/arrays/ops/take_single.rs +++ b/common/datavalues/src/arrays/ops/take_single.rs @@ -1,6 +1,8 @@ // Copyright 2020-2021 The Datafuse Authors. // // SPDX-License-Identifier: Apache-2.0. +use std::sync::Arc; + use common_arrow::arrow::array::Array; use common_arrow::arrow::array::ArrayRef; @@ -13,7 +15,7 @@ use crate::DFNumericType; use crate::DFUtf8Array; macro_rules! impl_take_random_get { - ($self:ident, $index:ident, $array_type:ty) => {{ + ($self:ident, $index:ident) => {{ // Safety: // index should be in bounds let arr = $self.downcast_ref(); @@ -26,7 +28,7 @@ macro_rules! impl_take_random_get { } macro_rules! impl_take_random_get_unchecked { - ($self:ident, $index:ident, $array_type:ty) => {{ + ($self:ident, $index:ident) => {{ let arr = $self.downcast_ref(); arr.value_unchecked($index) }}; @@ -39,12 +41,12 @@ where T: DFNumericType #[inline] fn get(&self, index: usize) -> Option { - unsafe { impl_take_random_get!(self, index, PrimitiveArray) } + unsafe { impl_take_random_get!(self, index) } } #[inline] unsafe fn get_unchecked(&self, index: usize) -> Self::Item { - impl_take_random_get_unchecked!(self, index, PrimitiveArray) + impl_take_random_get_unchecked!(self, index) } } @@ -71,12 +73,12 @@ impl TakeRandom for DFBooleanArray { fn get(&self, index: usize) -> Option { // Safety: // Out of bounds is checked and downcast is of correct type - unsafe { impl_take_random_get!(self, index, BooleanArray) } + unsafe { impl_take_random_get!(self, index) } } #[inline] unsafe fn get_unchecked(&self, index: usize) -> Self::Item { - impl_take_random_get_unchecked!(self, index, BooleanArray) + impl_take_random_get_unchecked!(self, index) } } @@ -87,12 +89,12 @@ impl<'a> TakeRandom for &'a DFUtf8Array { fn get(&self, index: usize) -> Option { // Safety: // Out of bounds is checked and downcast is of correct type - unsafe { impl_take_random_get!(self, index, StringArray) } + unsafe { impl_take_random_get!(self, index) } } #[inline] unsafe fn get_unchecked(&self, index: usize) -> Self::Item { - impl_take_random_get_unchecked!(self, index, StringArray) + impl_take_random_get_unchecked!(self, index) } } @@ -105,12 +107,12 @@ impl<'a> TakeRandomUtf8 for &'a DFUtf8Array { fn get(self, index: usize) -> Option { // Safety: // Out of bounds is checkedn and downcast is of correct type - unsafe { impl_take_random_get!(self, index, StringArray) } + unsafe { impl_take_random_get!(self, index) } } #[inline] unsafe fn get_unchecked(self, index: usize) -> Self::Item { - impl_take_random_get_unchecked!(self, index, StringArray) + impl_take_random_get_unchecked!(self, index) } } @@ -121,11 +123,16 @@ impl TakeRandom for DFListArray { fn get(&self, index: usize) -> Option { // Safety: // Out of bounds is checked and downcast is of correct type - unsafe { impl_take_random_get!(self, index, LargeListArray) } + let arr = self.downcast_ref(); + if arr.is_valid(index) { + return Some(Arc::from(arr.value(index))); + } + None } #[inline] unsafe fn get_unchecked(&self, index: usize) -> Self::Item { - impl_take_random_get_unchecked!(self, index, LargeListArray) + let arr = self.downcast_ref(); + return Arc::from(arr.value_unchecked(index)); } } diff --git a/common/datavalues/src/arrays/ops/to_values.rs b/common/datavalues/src/arrays/ops/to_values.rs index 8bd3a770f86f8..d69f1e900a36c 100644 --- a/common/datavalues/src/arrays/ops/to_values.rs +++ b/common/datavalues/src/arrays/ops/to_values.rs @@ -5,36 +5,12 @@ use std::fmt::Debug; use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::PrimitiveArray; +use common_arrow::arrow::array::ArrayRef; use common_exception::Result; use crate::arrays::DataArray; use crate::prelude::*; -use crate::DFBinaryArray; -use crate::DFBooleanArray; -use crate::DFListArray; -use crate::DFNullArray; -use crate::DFStructArray; -use crate::DFUtf8Array; -use crate::DataValue; -use crate::Date32Type; -use crate::Date64Type; -use crate::Float32Type; -use crate::Float64Type; -use crate::Int16Type; -use crate::Int32Type; -use crate::Int64Type; -use crate::Int8Type; -use crate::IntervalDayTimeType; -use crate::IntervalYearMonthType; -use crate::TimestampMicrosecondType; -use crate::TimestampMillisecondType; -use crate::TimestampNanosecondType; -use crate::TimestampSecondType; -use crate::UInt16Type; -use crate::UInt32Type; -use crate::UInt64Type; -use crate::UInt8Type; +use crate::*; /// This trait is used to compact a column into a Vec. /// It is mainly used for subquery execution. @@ -43,11 +19,12 @@ pub trait ToValues: Debug { fn to_values(&self) -> Result>; } -fn primitive_type_to_values_impl(array: &PrimitiveArray, f: F) -> Result> +fn primitive_type_to_values_impl(array: &DataArray, f: F) -> Result> where T: DFPrimitiveType, F: Fn(Option) -> DataValue, { + let array = array.downcast_ref(); let mut values = Vec::with_capacity(array.len()); if array.null_count() == 0 { @@ -68,109 +45,109 @@ where impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Int8) + primitive_type_to_values_impl(self, DataValue::Int8) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Int16) + primitive_type_to_values_impl(self, DataValue::Int16) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Int32) + primitive_type_to_values_impl(self, DataValue::Int32) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Int64) + primitive_type_to_values_impl(self, DataValue::Int64) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::UInt8) + primitive_type_to_values_impl(self, DataValue::UInt8) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::UInt16) + primitive_type_to_values_impl(self, DataValue::UInt16) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::UInt32) + primitive_type_to_values_impl(self, DataValue::UInt32) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::UInt64) + primitive_type_to_values_impl(self, DataValue::UInt64) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Float32) + primitive_type_to_values_impl(self, DataValue::Float32) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Float64) + primitive_type_to_values_impl(self, DataValue::Float64) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::IntervalDayTime) + primitive_type_to_values_impl(self, DataValue::IntervalDayTime) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::IntervalYearMonth) + primitive_type_to_values_impl(self, DataValue::IntervalYearMonth) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::TimestampSecond) + primitive_type_to_values_impl(self, DataValue::TimestampSecond) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::TimestampNanosecond) + primitive_type_to_values_impl(self, DataValue::TimestampNanosecond) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::TimestampMillisecond) + primitive_type_to_values_impl(self, DataValue::TimestampMillisecond) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::TimestampMicrosecond) + primitive_type_to_values_impl(self, DataValue::TimestampMicrosecond) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Date32) + primitive_type_to_values_impl(self, DataValue::Date32) } } impl ToValues for DataArray { fn to_values(&self) -> Result> { - primitive_type_to_values_impl(self.downcast_ref(), DataValue::Date64) + primitive_type_to_values_impl(self, DataValue::Date64) } } @@ -249,7 +226,7 @@ impl ToValues for DFListArray { let data_type = ele_type.data_type(); if array.null_count() == 0 { for index in 0..self.len() { - let list = array.value(index); + let list: ArrayRef = Arc::from(array.value(index)); let list_values = list.into_series().to_values()?; values.push(DataValue::List(Some(list_values), data_type.clone())) } @@ -258,7 +235,7 @@ impl ToValues for DFListArray { match array.is_null(index) { true => values.push(DataValue::List(None, data_type.clone())), false => { - let list = array.value(index); + let list: ArrayRef = Arc::from(array.value(index)); let list_values = list.into_series().to_values()?; values.push(DataValue::List(Some(list_values), data_type.clone())) } @@ -282,10 +259,10 @@ impl ToValues for DFStructArray { let mut values = Vec::with_capacity(self.len()); let array = self.downcast_ref(); - let mut columns_values = Vec::with_capacity(array.num_columns()); + let mut columns_values = Vec::with_capacity(array.fields().len()); - for column in array.columns_ref() { - let series = column.into_series(); + for column in array.values() { + let series = column.clone().into_series(); columns_values.push(series.to_values()?); } diff --git a/common/datavalues/src/arrays/ops/vec_hash.rs b/common/datavalues/src/arrays/ops/vec_hash.rs index 824f0a3c5d0d1..7a158fc6065ee 100644 --- a/common/datavalues/src/arrays/ops/vec_hash.rs +++ b/common/datavalues/src/arrays/ops/vec_hash.rs @@ -93,7 +93,7 @@ impl VecHash for DFFloat64Array { impl VecHash for DFBinaryArray { fn vec_hash(&self, hasher: DFHasher) -> Result { let binary_data = self.downcast_ref(); - let mut builder = PrimitiveArrayBuilder::::new(self.len()); + let mut builder = PrimitiveArrayBuilder::::with_capacity(self.len()); (0..self.len()).for_each(|index| { if self.is_null(index) { diff --git a/common/datavalues/src/arrays/trusted_len.rs b/common/datavalues/src/arrays/trusted_len.rs new file mode 100644 index 0000000000000..16e65726bcc29 --- /dev/null +++ b/common/datavalues/src/arrays/trusted_len.rs @@ -0,0 +1,96 @@ +// Copyright 2020-2021 The Datafuse Authors. +// +// SPDX-License-Identifier: Apache-2.0. + +use std::borrow::Borrow; + +use common_arrow::arrow::array::*; +use common_arrow::arrow::buffer::Buffer; +use common_arrow::arrow::trusted_len::TrustedLen; + +use super::DFAsRef; +use crate::prelude::*; +use crate::utils::NoNull; + +pub trait FromTrustedLenIterator: Sized { + fn from_iter_trusted_length>(iter: T) -> Self + where T::IntoIter: TrustedLen; +} + +impl FromTrustedLenIterator> for DataArray +where T: DFPrimitiveType +{ + fn from_iter_trusted_length>>(iter: I) -> Self { + let iter = iter.into_iter(); + + let arr = unsafe { + PrimitiveArray::from_trusted_len_iter_unchecked(iter).to(T::data_type().to_arrow()) + }; + Self::from_arrow_array(arr) + } +} + +// NoNull is only a wrapper needed for specialization +impl FromTrustedLenIterator for NoNull> +where T: DFPrimitiveType +{ + // We use AlignedVec because it is way faster than Arrows builder. We can do this because we + // know we don't have null values. + fn from_iter_trusted_length>(iter: I) -> Self { + let iter = iter.into_iter(); + let values = unsafe { Buffer::from_trusted_len_iter_unchecked(iter) }; + let arr = PrimitiveArray::from_data(T::data_type().to_arrow(), values, None); + + NoNull::new(DataArray::::from_arrow_array(arr)) + } +} +impl FromTrustedLenIterator for DFListArray +where Ptr: Borrow +{ + fn from_iter_trusted_length>(iter: I) -> Self { + let iter = iter.into_iter(); + iter.collect() + } +} + +impl FromTrustedLenIterator> for DFListArray +where Ptr: Borrow +{ + fn from_iter_trusted_length>>(iter: I) -> Self { + let iter = iter.into_iter(); + iter.collect() + } +} + +impl FromTrustedLenIterator> for DFBooleanArray { + fn from_iter_trusted_length>>(iter: I) -> Self + where I::IntoIter: TrustedLen { + let iter = iter.into_iter(); + let arr = BooleanArray::from_trusted_len_iter(iter); + Self::from_arrow_array(arr) + } +} + +impl FromTrustedLenIterator for DFBooleanArray { + fn from_iter_trusted_length>(iter: I) -> Self + where I::IntoIter: TrustedLen { + let iter = iter.into_iter(); + let arr = BooleanArray::from_trusted_len_values_iter(iter); + Self::from_arrow_array(arr) + } +} + +impl FromTrustedLenIterator for NoNull { + fn from_iter_trusted_length>(iter: I) -> Self { + let iter = iter.into_iter(); + iter.collect() + } +} +impl FromTrustedLenIterator for DFUtf8Array +where Ptr: DFAsRef +{ + fn from_iter_trusted_length>(iter: I) -> Self { + let iter = iter.into_iter(); + iter.collect() + } +} diff --git a/common/datavalues/src/arrays/upstream_traits.rs b/common/datavalues/src/arrays/upstream_traits.rs index d6d271327a430..dcd2ba3174b51 100644 --- a/common/datavalues/src/arrays/upstream_traits.rs +++ b/common/datavalues/src/arrays/upstream_traits.rs @@ -1,23 +1,22 @@ // Copyright 2020-2021 The Datafuse Authors. // // SPDX-License-Identifier: Apache-2.0. + //! Implementations of upstream traits for DataArray use std::borrow::Borrow; use std::borrow::Cow; use std::iter::FromIterator; use std::sync::Arc; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::BooleanArray; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::array::StringArray; +use common_arrow::arrow::array::*; use super::get_list_builder; use crate::arrays::DataArray; +use crate::prelude::AlignedVec; +use crate::prelude::LargeUtf8Array; use crate::series::Series; use crate::utils::get_iter_capacity; use crate::utils::NoNull; -use crate::vec::AlignedVec; use crate::DFBooleanArray; use crate::DFListArray; use crate::DFNumericType; @@ -32,7 +31,7 @@ where T: DFPrimitiveType fn from_iter>>(iter: I) -> Self { let iter = iter.into_iter(); - let arr: PrimitiveArray = match iter.size_hint() { + let arr: PrimitiveArray = match iter.size_hint() { (a, Some(b)) if a == b => { // 2021-02-07: ~40% faster than builder. // It is unsafe because we cannot be certain that the iterators length can be trusted. @@ -40,7 +39,7 @@ where T: DFPrimitiveType // somebody can create an iterator that incorrectly gives those bounds. // This will not lead to UB, but will panic. unsafe { - let arr = PrimitiveArray::from_trusted_len_iter(iter); + let arr = PrimitiveArray::from_trusted_len_iter_unchecked(iter); assert_eq!(arr.len(), a); arr } @@ -65,7 +64,7 @@ where T: DFPrimitiveType fn from_iter>(iter: I) -> Self { // 2021-02-07: aligned vec was ~2x faster than arrow collect. let iter = iter.into_iter(); - let mut av = AlignedVec::with_capacity_aligned(0); + let mut av = AlignedVec::with_capacity(0); av.extend(iter); NoNull::new(DataArray::new_from_aligned_vec(av)) } @@ -102,7 +101,7 @@ where Ptr: AsRef { fn from_iter>>(iter: I) -> Self { // 2021-02-07: this was ~30% faster than with the builder. - let arr = StringArray::from_iter(iter); + let arr = LargeUtf8Array::from_iter(iter); let array = Arc::new(arr) as ArrayRef; array.into() } @@ -121,7 +120,7 @@ impl FromIterator for DFUtf8Array where Ptr: DFAsRef { fn from_iter>(iter: I) -> Self { - let arr = StringArray::from_iter_values(iter); + let arr = LargeUtf8Array::from_iter_values(iter); let array = Arc::new(arr) as ArrayRef; array.into() @@ -145,13 +144,13 @@ impl From for Vec> { impl<'a> From<&'a DFBooleanArray> for Vec> { fn from(ca: &'a DFBooleanArray) -> Self { - ca.downcast_iter().collect() + ca.collect_values() } } impl From for Vec> { fn from(ca: DFBooleanArray) -> Self { - ca.downcast_iter().collect() + ca.collect_values() } } @@ -159,7 +158,7 @@ impl<'a, T> From<&'a DataArray> for Vec> where T: DFNumericType { fn from(ca: &'a DataArray) -> Self { - ca.downcast_iter().collect() + ca.collect_values() } } diff --git a/common/datavalues/src/columns/common.rs b/common/datavalues/src/columns/common.rs index d881e6dc84b1b..385a00adbfb0f 100644 --- a/common/datavalues/src/columns/common.rs +++ b/common/datavalues/src/columns/common.rs @@ -2,16 +2,8 @@ // // SPDX-License-Identifier: Apache-2.0. -use std::cmp::Ordering; - -use common_arrow::arrow::array::build_compare; -use common_arrow::arrow::array::make_array; use common_arrow::arrow::array::Array; -use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::array::DynComparator; -use common_arrow::arrow::array::MutableArrayData; -use common_arrow::arrow::compute; -use common_arrow::arrow::compute::SortOptions; +use common_arrow::arrow::compute::concat; use common_exception::ErrorCode; use common_exception::Result; @@ -27,40 +19,9 @@ impl DataColumnCommon { let dyn_arrays: Vec<&dyn Array> = arrays.iter().map(|arr| arr.as_ref()).collect(); - let array = compute::concat(&dyn_arrays)?; + let array: Arc = Arc::from(concat::concatenate(&dyn_arrays)?); Ok(array.into()) } - - pub fn merge_columns( - lhs: &DataColumn, - rhs: &DataColumn, - indices: &[bool], - ) -> Result { - let lhs = lhs.to_array()?; - let rhs = rhs.to_array()?; - - let result = - DataArrayMerge::merge_array(&lhs.get_array_ref(), &rhs.get_array_ref(), indices)?; - Ok(result.into()) - } - - pub fn merge_indices( - lhs: &[DataColumn], - rhs: &[DataColumn], - options: &[SortOptions], - limit: Option, - ) -> Result> { - let lhs: Vec = lhs - .iter() - .map(|s| s.get_array_ref()) - .collect::>>()?; - let rhs: Vec = rhs - .iter() - .map(|s| s.get_array_ref()) - .collect::>>()?; - - DataArrayMerge::merge_indices(&lhs, &rhs, options, limit) - } } impl DataColumn { @@ -169,161 +130,3 @@ impl DataColumn { Ok(()) } } - -struct DataArrayMerge; - -impl DataArrayMerge { - fn merge_array(lhs: &ArrayRef, rhs: &ArrayRef, indices: &[bool]) -> Result { - if lhs.data_type() != rhs.data_type() { - return Result::Err(ErrorCode::BadDataValueType( - "It is impossible to merge arrays of different data types.", - )); - } - - if lhs.len() + rhs.len() < indices.len() || indices.is_empty() { - return Result::Err(ErrorCode::BadDataArrayLength(format!( - "It is impossible to merge arrays with overflow indices, {}", - indices.len() - ))); - } - - let arrays = vec![lhs, rhs] - .iter() - .map(|a| a.data_ref()) - .collect::>(); - - let mut mutable = MutableArrayData::new(arrays, false, indices.len()); - let (mut left_next, mut right_next, mut last_is_left) = (0usize, 0usize, indices[0]); - - // tomb value - let extend_indices = [indices, &[false]].concat(); - - for (pos, &is_left) in extend_indices[1..].iter().enumerate() { - if is_left != last_is_left || pos + 1 == indices.len() { - if last_is_left { - mutable.extend(0, left_next, pos + 1 - right_next); - left_next = pos + 1 - right_next; - } else { - mutable.extend(1, right_next, pos + 1 - left_next); - right_next = pos + 1 - left_next; - } - last_is_left = is_left; - } - } - - Ok(make_array(mutable.freeze())) - } - - /// Given two sets of _ordered_ arrays, returns a bool vector denoting which of the items of the lhs and rhs are to pick from so that - /// if we were to sort-merge the lhs and rhs arrays together, they would all be sorted according to the `options`. - /// # Errors - /// This function errors when: - /// * `lhs.len() != rhs.len()` - /// * `lhs.len() == 0` - /// * `lhs.len() != options.len()` - /// * Arrays on `lhs` and `rhs` have no order relationship - pub fn merge_indices( - lhs: &[ArrayRef], - rhs: &[ArrayRef], - options: &[SortOptions], - limit: Option, - ) -> Result> { - if lhs.len() != rhs.len() { - return Result::Err(ErrorCode::BadDataArrayLength( - format!( - "Merge requires lhs and rhs to have the same number of arrays. lhs has {}, rhs has {}.", - lhs.len(), - rhs.len() - ) - )); - }; - if lhs.is_empty() { - return Result::Err(ErrorCode::BadDataArrayLength( - "Merge requires lhs to have at least 1 entry.", - )); - }; - if lhs.len() != options.len() { - return Result::Err(ErrorCode::BadDataArrayLength( - format!( - "Merge requires the number of sort options to equal number of columns. lhs has {} entries, options has {} entries", - lhs.len(), - options.len() - ) - )); - }; - - // prepare the comparison function between lhs and rhs arrays - let cmp = lhs - .iter() - .zip(rhs.iter()) - .map(|(l, r)| build_compare(l.as_ref(), r.as_ref())) - .collect::>>()?; - - // prepare a comparison function taking into account nulls and sort options - let cmp = |left, right| { - for c in 0..lhs.len() { - let descending = options[c].descending; - let null_first = options[c].nulls_first; - let mut result = match (lhs[c].is_valid(left), rhs[c].is_valid(right)) { - (true, true) => (cmp[c])(left, right), - (false, true) => { - if null_first { - Ordering::Less - } else { - Ordering::Greater - } - } - (true, false) => { - if null_first { - Ordering::Greater - } else { - Ordering::Less - } - } - (false, false) => Ordering::Equal, - }; - if descending { - result = result.reverse(); - }; - if result != Ordering::Equal { - // we found a relevant comparison => short-circuit and return it - return result; - } - } - Ordering::Equal - }; - - // the actual merge-sort code is from this point onwards - let mut left = 0; // Head of left pile. - let mut right = 0; // Head of right pile. - let max_left = lhs[0].len(); - let max_right = rhs[0].len(); - - let limits = match limit { - Some(limit) => limit.min(max_left + max_right), - _ => max_left + max_right, - }; - - let mut result = Vec::with_capacity(limits); - while left < max_left || right < max_right { - let order = match (left >= max_left, right >= max_right) { - (true, true) => break, - (false, true) => Ordering::Less, - (true, false) => Ordering::Greater, - (false, false) => (cmp)(left, right), - }; - let value = if order == Ordering::Less { - left += 1; - true - } else { - right += 1; - false - }; - result.push(value); - if result.len() >= limits { - break; - } - } - Ok(result) - } -} diff --git a/common/datavalues/src/data_array_filter.rs b/common/datavalues/src/data_array_filter.rs index 377fd2180a5c7..02184a56a0e19 100644 --- a/common/datavalues/src/data_array_filter.rs +++ b/common/datavalues/src/data_array_filter.rs @@ -2,7 +2,10 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow; +use std::ops::BitAnd; + +use common_arrow::arrow::array::*; +use common_arrow::arrow::compute::filter::build_filter; use common_exception::Result; use crate::prelude::*; @@ -18,16 +21,32 @@ impl DataArrayFilter { if predicate.null_count() > 0 { // this greatly simplifies subsequent filtering code // now we only have a boolean mask to deal with - let predicate = arrow::compute::prep_null_mask_filter(predicate.downcast_ref()); - let predicate_array = DFBooleanArray::from_arrow_array(predicate); - return Self::filter_batch_array(array, &predicate_array); + let predicate = Self::remove_null_filter(predicate); + return Self::filter_batch_array(array, &predicate); } - let filter = arrow::compute::build_filter(predicate.downcast_ref())?; - let filtered_arrays = array + let filter = build_filter(predicate.downcast_ref())?; + let filtered_arrays: Vec = array .iter() - .map(|a| arrow::array::make_array(filter(a.get_array_ref().data())).into_series()) + .map(|a| { + let c = a.get_array_ref(); + let c = filter(c.as_ref()); + let c: Arc = Arc::from(c); + c.into_series() + }) .collect(); + Ok(filtered_arrays) } + + /// Remove null values by do a bitmask AND operation with null bits and the boolean bits. + fn remove_null_filter(filter: &DFBooleanArray) -> DFBooleanArray { + let array = filter.downcast_ref(); + let mask = array.values(); + if let Some(v) = array.validity() { + let mask = mask.bitand(v); + return DFBooleanArray::from_arrow_array(BooleanArray::from_data(mask, None)); + } + filter.clone() + } } diff --git a/common/datavalues/src/data_df_type.rs b/common/datavalues/src/data_df_type.rs index 59715c41ec7a5..cb7cea5b94503 100644 --- a/common/datavalues/src/data_df_type.rs +++ b/common/datavalues/src/data_df_type.rs @@ -2,15 +2,11 @@ // // SPDX-License-Identifier: Apache-2.0. -use common_arrow::arrow::datatypes as arrow_data_types; -use common_arrow::arrow::datatypes::ArrowNumericType; -use common_arrow::arrow::datatypes::ArrowPrimitiveType; -use common_arrow::arrow::datatypes::IntervalUnit; -use common_arrow::arrow::datatypes::TimeUnit; +use common_arrow::arrow::types::NativeType; use crate::arrays::DataArray; +use crate::data_type::*; use crate::DataField; -use crate::DataType; pub trait DFDataType: Send + Sync { fn data_type() -> DataType; @@ -18,6 +14,8 @@ pub trait DFDataType: Send + Sync { macro_rules! impl_df_datatype { ($ca:ident, $variant:ident) => { + pub struct $ca {} + impl DFDataType for $ca { fn data_type() -> DataType { DataType::$variant @@ -26,67 +24,6 @@ macro_rules! impl_df_datatype { }; } -pub type UInt8Type = arrow_data_types::UInt8Type; -pub type UInt16Type = arrow_data_types::UInt16Type; -pub type UInt32Type = arrow_data_types::UInt32Type; -pub type UInt64Type = arrow_data_types::UInt64Type; -pub type Int8Type = arrow_data_types::Int8Type; -pub type Int16Type = arrow_data_types::Int16Type; -pub type Int32Type = arrow_data_types::Int32Type; -pub type Int64Type = arrow_data_types::Int64Type; -pub type Float32Type = arrow_data_types::Float32Type; -pub type Float64Type = arrow_data_types::Float64Type; - -pub type BooleanType = arrow_data_types::BooleanType; - -pub type Date32Type = arrow_data_types::Date32Type; -pub type Date64Type = arrow_data_types::Date64Type; - -pub type TimestampSecondType = arrow_data_types::TimestampSecondType; -pub type TimestampMillisecondType = arrow_data_types::TimestampMillisecondType; -pub type TimestampMicrosecondType = arrow_data_types::TimestampMicrosecondType; -pub type TimestampNanosecondType = arrow_data_types::TimestampNanosecondType; - -pub type IntervalYearMonthType = arrow_data_types::IntervalYearMonthType; -pub type IntervalDayTimeType = arrow_data_types::IntervalDayTimeType; - -pub struct NullType; -pub struct Utf8Type; -pub struct ListType; - -pub struct StructType; -pub struct BinaryType; - -pub type DFNullArray = DataArray; -pub type DFInt8Array = DataArray; -pub type DFUInt8Array = DataArray; -pub type DFInt16Array = DataArray; -pub type DFUInt16Array = DataArray; -pub type DFInt32Array = DataArray; -pub type DFUInt32Array = DataArray; -pub type DFInt64Array = DataArray; -pub type DFUInt64Array = DataArray; - -pub type DFBooleanArray = DataArray; - -pub type DFFloat32Array = DataArray; -pub type DFFloat64Array = DataArray; - -pub type DFUtf8Array = DataArray; -pub type DFListArray = DataArray; -pub type DFStructArray = DataArray; -pub type DFBinaryArray = DataArray; - -pub type DFDate32Array = DataArray; -pub type DFDate64Array = DataArray; - -pub type DFTimestampSecondArray = DataArray; -pub type DFTimestampMillisecondArray = DataArray; -pub type DFTimestampMicrosecondArray = DataArray; -pub type DFTimestampNanosecondArray = DataArray; -pub type DFIntervalYearMonthArray = DataArray; -pub type DFIntervalDayTimeArray = DataArray; - impl_df_datatype!(UInt8Type, UInt8); impl_df_datatype!(UInt16Type, UInt16); impl_df_datatype!(UInt32Type, UInt32); @@ -101,48 +38,52 @@ impl_df_datatype!(BooleanType, Boolean); impl_df_datatype!(Date32Type, Date32); impl_df_datatype!(Date64Type, Date64); +impl_df_datatype!(Utf8Type, Utf8); +impl_df_datatype!(NullType, Null); +impl_df_datatype!(BinaryType, Binary); + +pub struct TimestampSecondType; impl DFDataType for TimestampSecondType { fn data_type() -> DataType { DataType::Timestamp(TimeUnit::Second, None) } } +pub struct TimestampMillisecondType; impl DFDataType for TimestampMillisecondType { fn data_type() -> DataType { DataType::Timestamp(TimeUnit::Millisecond, None) } } +pub struct TimestampMicrosecondType; impl DFDataType for TimestampMicrosecondType { fn data_type() -> DataType { DataType::Timestamp(TimeUnit::Microsecond, None) } } +pub struct TimestampNanosecondType; impl DFDataType for TimestampNanosecondType { fn data_type() -> DataType { DataType::Timestamp(TimeUnit::Nanosecond, None) } } +pub struct IntervalYearMonthType; impl DFDataType for IntervalYearMonthType { fn data_type() -> DataType { DataType::Interval(IntervalUnit::YearMonth) } } - +pub struct IntervalDayTimeType; impl DFDataType for IntervalDayTimeType { fn data_type() -> DataType { DataType::Interval(IntervalUnit::DayTime) } } -impl DFDataType for Utf8Type { - fn data_type() -> DataType { - DataType::Utf8 - } -} - +pub struct ListType; impl DFDataType for ListType { fn data_type() -> DataType { // null as we cannot no anything without self. @@ -150,92 +91,135 @@ impl DFDataType for ListType { } } -impl DFDataType for NullType { +pub struct StructType; +impl DFDataType for StructType { fn data_type() -> DataType { - DataType::Null + // null as we cannot no anything without self. + DataType::Struct(vec![DataField::new("", DataType::Null, true)]) } } -impl DFDataType for BinaryType { - fn data_type() -> DataType { - // null as we cannot no anything without self. - DataType::Binary - } +pub type DFNullArray = DataArray; +pub type DFInt8Array = DataArray; +pub type DFUInt8Array = DataArray; +pub type DFInt16Array = DataArray; +pub type DFUInt16Array = DataArray; +pub type DFInt32Array = DataArray; +pub type DFUInt32Array = DataArray; +pub type DFInt64Array = DataArray; +pub type DFUInt64Array = DataArray; + +pub type DFBooleanArray = DataArray; + +pub type DFFloat32Array = DataArray; +pub type DFFloat64Array = DataArray; + +pub type DFUtf8Array = DataArray; +pub type DFListArray = DataArray; +pub type DFStructArray = DataArray; +pub type DFBinaryArray = DataArray; + +pub type DFDate32Array = DataArray; +pub type DFDate64Array = DataArray; + +pub type DFTimestampSecondArray = DataArray; +pub type DFTimestampMillisecondArray = DataArray; +pub type DFTimestampMicrosecondArray = DataArray; +pub type DFTimestampNanosecondArray = DataArray; +pub type DFIntervalYearMonthArray = DataArray; +pub type DFIntervalDayTimeArray = DataArray; + +pub trait DFPrimitiveType: Send + Sync + DFDataType + 'static { + type Native: NativeType; } -impl DFDataType for StructType { - fn data_type() -> DataType { - // null as we cannot no anything without self. - DataType::Struct(vec![DataField::new("", DataType::Null, true)]) - } +macro_rules! impl_primitive { + ($ca:ident, $native:ident) => { + impl DFPrimitiveType for $ca { + type Native = $native; + } + }; +} + +impl_primitive!(UInt8Type, u8); +impl_primitive!(UInt16Type, u16); +impl_primitive!(UInt32Type, u32); +impl_primitive!(UInt64Type, u64); +impl_primitive!(Int8Type, i8); +impl_primitive!(Int16Type, i16); +impl_primitive!(Int32Type, i32); +impl_primitive!(Int64Type, i64); +impl_primitive!(Float32Type, f32); +impl_primitive!(Float64Type, f64); + +impl_primitive!(Date32Type, i32); +impl_primitive!(Date64Type, i64); + +impl_primitive!(TimestampSecondType, i64); +impl_primitive!(TimestampMillisecondType, i64); +impl_primitive!(TimestampMicrosecondType, i64); +impl_primitive!(TimestampNanosecondType, i64); + +impl_primitive!(IntervalYearMonthType, i32); +impl_primitive!(IntervalDayTimeType, i64); + +pub trait DFNumericType: DFPrimitiveType {} + +macro_rules! impl_numeric { + ($ca:ident, $native:ident) => { + impl DFNumericType for $ca {} + }; } -pub trait DFPrimitiveType: ArrowPrimitiveType + Send + Sync + DFDataType {} - -impl DFPrimitiveType for UInt8Type {} -impl DFPrimitiveType for UInt16Type {} -impl DFPrimitiveType for UInt32Type {} -impl DFPrimitiveType for UInt64Type {} -impl DFPrimitiveType for Int8Type {} -impl DFPrimitiveType for Int16Type {} -impl DFPrimitiveType for Int32Type {} -impl DFPrimitiveType for Int64Type {} -impl DFPrimitiveType for Float32Type {} -impl DFPrimitiveType for Float64Type {} -impl DFPrimitiveType for Date32Type {} -impl DFPrimitiveType for Date64Type {} - -impl DFPrimitiveType for TimestampSecondType {} -impl DFPrimitiveType for TimestampMillisecondType {} -impl DFPrimitiveType for TimestampMicrosecondType {} -impl DFPrimitiveType for TimestampNanosecondType {} - -impl DFPrimitiveType for IntervalYearMonthType {} -impl DFPrimitiveType for IntervalDayTimeType {} - -pub trait DFNumericType: DFPrimitiveType + ArrowNumericType {} - -impl DFNumericType for UInt8Type {} -impl DFNumericType for UInt16Type {} -impl DFNumericType for UInt32Type {} -impl DFNumericType for UInt64Type {} - -impl DFNumericType for Int8Type {} -impl DFNumericType for Int16Type {} -impl DFNumericType for Int32Type {} -impl DFNumericType for Int64Type {} -impl DFNumericType for Float32Type {} -impl DFNumericType for Float64Type {} -impl DFNumericType for Date32Type {} -impl DFNumericType for Date64Type {} - -impl DFNumericType for TimestampSecondType {} -impl DFNumericType for TimestampMillisecondType {} -impl DFNumericType for TimestampMicrosecondType {} -impl DFNumericType for TimestampNanosecondType {} - -impl DFNumericType for IntervalYearMonthType {} -impl DFNumericType for IntervalDayTimeType {} +impl_numeric!(UInt8Type, u8); +impl_numeric!(UInt16Type, u16); +impl_numeric!(UInt32Type, u32); +impl_numeric!(UInt64Type, u64); +impl_numeric!(Int8Type, i8); +impl_numeric!(Int16Type, i16); +impl_numeric!(Int32Type, i32); +impl_numeric!(Int64Type, i64); +impl_numeric!(Float32Type, f32); +impl_numeric!(Float64Type, f64); + +impl_numeric!(Date32Type, i32); +impl_numeric!(Date64Type, i64); + +impl_numeric!(TimestampSecondType, i64); +impl_numeric!(TimestampMillisecondType, i64); +impl_numeric!(TimestampMicrosecondType, i32); +impl_numeric!(TimestampNanosecondType, i64); + +impl_numeric!(IntervalYearMonthType, i32); +impl_numeric!(IntervalDayTimeType, i64); pub trait DFIntegerType: DFNumericType {} -impl DFIntegerType for UInt8Type {} -impl DFIntegerType for UInt16Type {} -impl DFIntegerType for UInt32Type {} -impl DFIntegerType for UInt64Type {} -impl DFIntegerType for Int8Type {} -impl DFIntegerType for Int16Type {} -impl DFIntegerType for Int32Type {} -impl DFIntegerType for Int64Type {} -impl DFIntegerType for Date32Type {} -impl DFIntegerType for Date64Type {} - -impl DFIntegerType for TimestampSecondType {} -impl DFIntegerType for TimestampMillisecondType {} -impl DFIntegerType for TimestampMicrosecondType {} -impl DFIntegerType for TimestampNanosecondType {} - -impl DFIntegerType for IntervalYearMonthType {} -impl DFIntegerType for IntervalDayTimeType {} + +macro_rules! impl_integer { + ($ca:ident, $native:ident) => { + impl DFIntegerType for $ca {} + }; +} + +impl_integer!(UInt8Type, u8); +impl_integer!(UInt16Type, u16); +impl_integer!(UInt32Type, u32); +impl_integer!(UInt64Type, u64); +impl_integer!(Int8Type, i8); +impl_integer!(Int16Type, i16); +impl_integer!(Int32Type, i32); +impl_integer!(Int64Type, i64); + +impl_integer!(Date32Type, i32); +impl_integer!(Date64Type, i64); + +impl_integer!(TimestampSecondType, i64); +impl_integer!(TimestampMillisecondType, i64); +impl_integer!(TimestampMicrosecondType, i32); +impl_integer!(TimestampNanosecondType, i64); + +impl_integer!(IntervalYearMonthType, i32); +impl_integer!(IntervalDayTimeType, i64); pub trait DFFloatType: DFNumericType {} impl DFFloatType for Float32Type {} diff --git a/common/datavalues/src/data_type.rs b/common/datavalues/src/data_type.rs index a6bf6565a0eda..c96ffac91fe3d 100644 --- a/common/datavalues/src/data_type.rs +++ b/common/datavalues/src/data_type.rs @@ -5,8 +5,8 @@ use core::fmt; use common_arrow::arrow::datatypes::DataType as ArrowDataType; -use common_arrow::arrow::datatypes::IntervalUnit; -use common_arrow::arrow::datatypes::TimeUnit; +use common_arrow::arrow::datatypes::IntervalUnit as ArrowIntervalUnit; +use common_arrow::arrow::datatypes::TimeUnit as ArrowTimeUnit; use crate::DataField; @@ -40,6 +40,52 @@ pub enum DataType { Binary, } +#[derive( + serde::Serialize, serde::Deserialize, Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord, +)] +pub enum TimeUnit { + /// Time in seconds. + Second, + /// Time in milliseconds. + Millisecond, + /// Time in microseconds. + Microsecond, + /// Time in nanoseconds. + Nanosecond, +} + +impl TimeUnit { + pub fn to_arrow(&self) -> ArrowTimeUnit { + unsafe { std::mem::transmute(self.clone()) } + } + + pub fn from_arrow(iu: &ArrowTimeUnit) -> Self { + unsafe { std::mem::transmute(iu.clone()) } + } +} + +/// YEAR_MONTH or DAY_TIME interval in SQL style. +#[derive( + serde::Serialize, serde::Deserialize, Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord, +)] +pub enum IntervalUnit { + /// Indicates the number of elapsed whole months, stored as 4-byte integers. + YearMonth, + /// Indicates the number of elapsed days and milliseconds, + /// stored as 2 contiguous 32-bit integers (8-bytes in total). + DayTime, +} + +impl IntervalUnit { + pub fn to_arrow(&self) -> ArrowIntervalUnit { + unsafe { std::mem::transmute(self.clone()) } + } + + pub fn from_arrow(iu: &ArrowIntervalUnit) -> Self { + unsafe { std::mem::transmute(iu.clone()) } + } +} + impl DataType { pub fn to_arrow(&self) -> ArrowDataType { use DataType::*; @@ -56,17 +102,17 @@ impl DataType { Int64 => ArrowDataType::Int64, Float32 => ArrowDataType::Float32, Float64 => ArrowDataType::Float64, - Utf8 => ArrowDataType::Utf8, + Utf8 => ArrowDataType::LargeUtf8, Date32 => ArrowDataType::Date32, Date64 => ArrowDataType::Date64, - Timestamp(tu, f) => ArrowDataType::Timestamp(tu.clone(), f.clone()), - Interval(tu) => ArrowDataType::Interval(tu.clone()), - List(dt) => ArrowDataType::List(Box::new(dt.to_arrow())), + Timestamp(tu, f) => ArrowDataType::Timestamp(tu.to_arrow(), f.clone()), + Interval(tu) => ArrowDataType::Interval(tu.to_arrow()), + List(dt) => ArrowDataType::LargeList(Box::new(dt.to_arrow())), Struct(fs) => { let arrows_fields = fs.iter().map(|f| f.to_arrow()).collect(); ArrowDataType::Struct(arrows_fields) } - Binary => ArrowDataType::Binary, + Binary => ArrowDataType::LargeBinary, } } } @@ -93,38 +139,26 @@ impl From<&ArrowDataType> for DataType { ArrowDataType::Boolean => DataType::Boolean, ArrowDataType::Float32 => DataType::Float32, ArrowDataType::Float64 => DataType::Float64, - ArrowDataType::List(f) => { + ArrowDataType::List(f) | ArrowDataType::LargeList(f) => { let f: DataField = (f.as_ref()).into(); DataType::List(Box::new(f)) } ArrowDataType::Date32 => DataType::Date32, ArrowDataType::Date64 => DataType::Date64, - ArrowDataType::Timestamp(TimeUnit::Second, f) => { - DataType::Timestamp(TimeUnit::Second, f.clone()) - } - ArrowDataType::Timestamp(TimeUnit::Millisecond, f) => { - DataType::Timestamp(TimeUnit::Millisecond, f.clone()) + ArrowDataType::Timestamp(tu, f) => { + DataType::Timestamp(TimeUnit::from_arrow(tu), f.clone()) } - ArrowDataType::Timestamp(TimeUnit::Microsecond, f) => { - DataType::Timestamp(TimeUnit::Microsecond, f.clone()) - } - ArrowDataType::Timestamp(TimeUnit::Nanosecond, f) => { - DataType::Timestamp(TimeUnit::Nanosecond, f.clone()) - } - ArrowDataType::Interval(IntervalUnit::YearMonth) => { - DataType::Interval(IntervalUnit::YearMonth) - } - ArrowDataType::Interval(IntervalUnit::DayTime) => { - DataType::Interval(IntervalUnit::DayTime) - } + ArrowDataType::Interval(fu) => DataType::Interval(IntervalUnit::from_arrow(fu)), - ArrowDataType::Utf8 => DataType::Utf8, - ArrowDataType::Binary => DataType::Binary, + ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 => DataType::Utf8, + ArrowDataType::Binary | ArrowDataType::LargeBinary => DataType::Binary, // this is safe, because we define the datatype firstly - _ => unimplemented!(), + _ => { + unimplemented!() + } } } } diff --git a/common/datavalues/src/data_value.rs b/common/datavalues/src/data_value.rs index 741b00461fde4..197b8fb3c2144 100644 --- a/common/datavalues/src/data_value.rs +++ b/common/datavalues/src/data_value.rs @@ -6,27 +6,28 @@ // See notice.md use std::fmt; -use std::iter::repeat; use std::ops::Deref; use std::sync::Arc; use common_arrow::arrow::array::*; use common_arrow::arrow::datatypes::Field as ArrowField; -use common_arrow::arrow::datatypes::IntervalUnit; -use common_arrow::arrow::datatypes::TimeUnit; use common_exception::ErrorCode; use common_exception::Result; use common_io::prelude::*; -use serde::Deserialize; -use serde::Serialize; +use crate::arrays::ListBooleanArrayBuilder; +use crate::arrays::ListBuilderTrait; +use crate::arrays::ListPrimitiveArrayBuilder; +use crate::arrays::ListUtf8ArrayBuilder; +use crate::data_type::*; +use crate::prelude::*; use crate::series::IntoSeries; use crate::series::Series; +use crate::DFInt8Array; use crate::DataField; -use crate::DataType; /// A specific value of a data type. -#[derive(Serialize, Deserialize, Clone, PartialEq)] +#[derive(serde::Serialize, serde::Deserialize, Clone, PartialEq)] pub enum DataValue { /// Base type. Null, @@ -141,159 +142,108 @@ impl DataValue { self.to_series_with_size(1) } - pub fn to_arrow_array_with_size(&self, size: usize) -> Result { + pub fn to_series_with_size(&self, size: usize) -> Result { match self { - DataValue::Null => Ok(Arc::new(NullArray::new(size))), - DataValue::Boolean(e) => match e { - Some(v) => Ok(Arc::new(BooleanArray::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Boolean, size)), - }, - DataValue::Int8(e) => match e { - Some(v) => Ok(Arc::new(Int8Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Int8, size)), - }, - DataValue::Int16(e) => match e { - Some(v) => Ok(Arc::new(Int16Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Int16, size)), - }, - DataValue::Int32(e) => match e { - Some(v) => Ok(Arc::new(Int32Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Int32, size)), - }, - DataValue::Int64(e) => match e { - Some(v) => Ok(Arc::new(Int64Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Int64, size)), - }, - DataValue::UInt8(e) => match e { - Some(v) => Ok(Arc::new(UInt8Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::UInt8, size)), - }, - DataValue::UInt16(e) => match e { - Some(v) => Ok(Arc::new(UInt16Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::UInt16, size)), - }, - DataValue::UInt32(e) => match e { - Some(v) => Ok(Arc::new(UInt32Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::UInt32, size)), - }, - DataValue::UInt64(e) => match e { - Some(v) => Ok(Arc::new(UInt64Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::UInt64, size)), - }, - DataValue::Float32(e) => match e { - Some(v) => Ok(Arc::new(Float32Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Float32, size)), - }, - DataValue::Float64(e) => match e { - Some(v) => Ok(Arc::new(Float64Array::from(vec![*v; size])) as ArrayRef), - None => Ok(new_null_array_by_type(&DataType::Float64, size)), - }, - DataValue::Utf8(e) => match e { - Some(v) => Ok(Arc::new(StringArray::from(vec![v.deref(); size]))), - None => Ok(new_null_array_by_type(&DataType::Utf8, size)), - }, - DataValue::Binary(e) => match e { - Some(v) => Ok(Arc::new(BinaryArray::from(vec![v.deref(); size]))), - None => Ok(new_null_array_by_type(&DataType::Binary, size)), - }, - DataValue::Date32(e) => match e { - Some(value) => Ok(Arc::new(Date32Array::from_value(*value, size))), - None => Ok(new_null_array_by_type(&DataType::Date32, size)), - }, - DataValue::Date64(e) => match e { - Some(value) => Ok(Arc::new(Date64Array::from_value(*value, size))), - None => Ok(new_null_array_by_type(&DataType::Date64, size)), - }, - DataValue::TimestampSecond(e) => match e { - Some(value) => Ok(Arc::new(TimestampSecondArray::from_iter_values( - repeat(*value).take(size), - ))), - None => Ok(new_null_array_by_type( - &DataType::Timestamp(TimeUnit::Second, None), - size, - )), - }, - DataValue::TimestampMillisecond(e) => match e { - Some(value) => Ok(Arc::new(TimestampMillisecondArray::from_iter_values( - repeat(*value).take(size), - ))), - None => Ok(new_null_array_by_type( - &DataType::Timestamp(TimeUnit::Millisecond, None), - size, - )), - }, - DataValue::TimestampMicrosecond(e) => match e { - Some(value) => Ok(Arc::new(TimestampMicrosecondArray::from_value( - *value, size, - ))), - None => Ok(new_null_array_by_type( - &DataType::Timestamp(TimeUnit::Microsecond, None), - size, - )), - }, - DataValue::TimestampNanosecond(e) => match e { - Some(value) => Ok(Arc::new(TimestampNanosecondArray::from_value(*value, size))), - None => Ok(new_null_array_by_type( - &DataType::Timestamp(TimeUnit::Nanosecond, None), - size, - )), - }, - DataValue::IntervalDayTime(e) => match e { - Some(value) => Ok(Arc::new(IntervalDayTimeArray::from_value(*value, size))), - None => Ok(new_null_array_by_type( - &DataType::Interval(IntervalUnit::DayTime), - size, - )), + DataValue::Null => { + let array = Arc::new(NullArray::new_null(size)) as ArrayRef; + let array: DFNullArray = array.into(); + Ok(array.into_series()) + } + DataValue::Boolean(values) => Ok(build_constant_series! {DFBooleanArray, values, size}), + DataValue::Int8(values) => Ok(build_constant_series! {DFInt8Array, values, size}), + DataValue::Int16(values) => Ok(build_constant_series! {DFInt16Array, values, size}), + DataValue::Int32(values) => Ok(build_constant_series! {DFInt32Array, values, size}), + DataValue::Int64(values) => Ok(build_constant_series! {DFInt64Array, values, size}), + DataValue::UInt8(values) => Ok(build_constant_series! {DFUInt8Array, values, size}), + DataValue::UInt16(values) => Ok(build_constant_series! {DFUInt16Array, values, size}), + DataValue::UInt32(values) => Ok(build_constant_series! {DFUInt32Array, values, size}), + DataValue::UInt64(values) => Ok(build_constant_series! {DFUInt64Array, values, size}), + DataValue::Float32(values) => Ok(build_constant_series! {DFFloat32Array, values, size}), + DataValue::Float64(values) => Ok(build_constant_series! {DFFloat64Array, values, size}), + + DataValue::Utf8(values) => match values { + None => Ok(DFUtf8Array::full_null(size).into_series()), + Some(v) => Ok(DFUtf8Array::full(v.deref(), size).into_series()), }, - DataValue::IntervalYearMonth(e) => match e { - Some(value) => Ok(Arc::new(IntervalYearMonthArray::from_value(*value, size))), - None => Ok(new_null_array_by_type( - &DataType::Interval(IntervalUnit::YearMonth), - size, - )), + + DataValue::Binary(values) => match values { + None => Ok(DFBinaryArray::full_null(size).into_series()), + Some(v) => Ok(DFBinaryArray::full(v.deref(), size).into_series()), }, + DataValue::List(values, data_type) => match data_type { - DataType::Int8 => Ok(Arc::new(build_list!(Int8Builder, Int8, values, size))), - DataType::Int16 => Ok(Arc::new(build_list!(Int16Builder, Int16, values, size))), - DataType::Int32 => Ok(Arc::new(build_list!(Int32Builder, Int32, values, size))), - DataType::Int64 => Ok(Arc::new(build_list!(Int64Builder, Int64, values, size))), - DataType::UInt8 => Ok(Arc::new(build_list!(UInt8Builder, UInt8, values, size))), - DataType::UInt16 => Ok(Arc::new(build_list!(UInt16Builder, UInt16, values, size))), - DataType::UInt32 => Ok(Arc::new(build_list!(UInt32Builder, UInt32, values, size))), - DataType::UInt64 => Ok(Arc::new(build_list!(UInt64Builder, UInt64, values, size))), - DataType::Float32 => { - Ok(Arc::new(build_list!(Float32Builder, Float32, values, size))) + DataType::Int8 => build_list_series! {Int8Type, values, size, data_type }, + DataType::Int16 => build_list_series! {Int16Type, values, size, data_type }, + DataType::Int32 => build_list_series! {Int32Type, values, size, data_type }, + DataType::Int64 => build_list_series! {Int64Type, values, size, data_type }, + + DataType::UInt8 => build_list_series! {UInt8Type, values, size, data_type }, + DataType::UInt16 => build_list_series! {UInt16Type, values, size, data_type }, + DataType::UInt32 => build_list_series! {UInt32Type, values, size, data_type }, + DataType::UInt64 => build_list_series! {UInt64Type, values, size, data_type }, + + DataType::Float32 => build_list_series! {Float32Type, values, size, data_type }, + DataType::Float64 => build_list_series! {Float64Type, values, size, data_type }, + + DataType::Boolean => { + let mut builder = ListBooleanArrayBuilder::with_capacity(0, size); + match values { + Some(v) => { + let series = DataValue::try_into_data_array(v, data_type)?; + (0..size).for_each(|_| { + builder.append_series(&series); + }); + } + None => (0..size).for_each(|_| { + builder.append_null(); + }), + } + Ok(builder.finish().into_series()) } - DataType::Float64 => { - Ok(Arc::new(build_list!(Float64Builder, Float64, values, size))) + DataType::Utf8 => { + let mut builder = ListUtf8ArrayBuilder::with_capacity(0, size); + match values { + Some(v) => { + let series = DataValue::try_into_data_array(v, data_type)?; + (0..size).for_each(|_| { + builder.append_series(&series); + }); + } + None => (0..size).for_each(|_| { + builder.append_null(); + }), + } + Ok(builder.finish().into_series()) } - DataType::Utf8 => Ok(Arc::new(build_list!(StringBuilder, Utf8, values, size))), other => Result::Err(ErrorCode::BadDataValueType(format!( "Unexpected type:{} for DataValue List", other ))), }, DataValue::Struct(v) => { - let mut array = vec![]; + let mut arrays = vec![]; + let mut fields = vec![]; for (i, x) in v.iter().enumerate() { - let val_array = x.to_arrow_array_with_size(1)?; - array.push(( - ArrowField::new( - format!("item_{}", i).as_str(), - val_array.data_type().clone(), - false, - ), - val_array as ArrayRef, + let xseries = x.to_series_with_size(size)?; + let val_array = xseries.get_array_ref(); + + fields.push(ArrowField::new( + format!("item_{}", i).as_str(), + val_array.data_type().clone(), + false, )); + + arrays.push(val_array); } - Ok(Arc::new(StructArray::from(array))) + let r = Arc::new(StructArray::from_data(fields, arrays, None)) as ArrayRef; + Ok(r.into_series()) } - } - } - pub fn to_series_with_size(&self, size: usize) -> Result { - let array = self.to_arrow_array_with_size(size)?; - Ok(array.into_series()) + other => Result::Err(ErrorCode::BadDataValueType(format!( + "Unexpected type:{} for DataValue", + other + ))), + } } pub fn to_values(&self, size: usize) -> Result> { @@ -335,11 +285,6 @@ impl DataValue { } } -#[inline] -fn new_null_array_by_type(data_type: &DataType, length: usize) -> ArrayRef { - new_null_array(&data_type.to_arrow(), length) -} - // Did not use std::convert:TryFrom // Because we do not need custom type error. pub trait DFTryFrom: Sized { diff --git a/common/datavalues/src/lib.rs b/common/datavalues/src/lib.rs index 732de0c91d395..9fe954fbb93e6 100644 --- a/common/datavalues/src/lib.rs +++ b/common/datavalues/src/lib.rs @@ -29,10 +29,6 @@ mod data_value_operator; mod data_value_ops; #[allow(dead_code)] mod utils; -mod vec; - -#[cfg(test)] -mod vec_test; pub mod arrays; pub mod columns; @@ -55,4 +51,3 @@ pub use data_value::DataValue; pub use data_value::DataValueRef; pub use data_value_arithmetic::*; pub use data_value_operator::*; -pub use vec::*; diff --git a/common/datavalues/src/macros.rs b/common/datavalues/src/macros.rs index 4ef0c2f180b9d..37065e1a09250 100644 --- a/common/datavalues/src/macros.rs +++ b/common/datavalues/src/macros.rs @@ -114,52 +114,37 @@ macro_rules! std_to_data_value { }; } -macro_rules! build_list { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr) => {{ - use common_arrow::arrow::datatypes::DataType as ArrowDataType; - use common_arrow::arrow::datatypes::Field as ArrowField; +macro_rules! build_constant_series { + ($ARRAY: ident, $VALUES: expr, $SIZE: expr) => { match $VALUES { - // the return on the macro is necessary, to short-circuit and return ArrayRef - None => { - return Ok(common_arrow::arrow::array::new_null_array( - &ArrowDataType::List(Box::new(ArrowField::new( - "item", - ArrowDataType::$SCALAR_TY, - true, - ))), - $SIZE, - )) - } - Some(values) => { - let mut builder = ListBuilder::new($VALUE_BUILDER_TY::new(values.len())); + Some(v) => $ARRAY::full(*v, $SIZE).into_series(), + None => $ARRAY::full_null($SIZE).into_series(), + } + }; +} - for _ in 0..$SIZE { - for scalar_value in values { - match scalar_value { - DataValue::$SCALAR_TY(Some(v)) => { - builder.values().append_value(v.clone()).unwrap() - } - DataValue::$SCALAR_TY(None) => { - builder.values().append_null().unwrap(); - } - _ => { - return Result::Err(ErrorCode::BadDataValueType( - "Incompatible DataValue for list", - )) - } - }; - } - builder.append(true).unwrap(); - } - builder.finish() +macro_rules! build_list_series { + ($TYPE:ident, $VALUES:expr, $SIZE:expr, $D_TYPE: expr) => {{ + type B = ListPrimitiveArrayBuilder<$TYPE>; + let mut builder = B::with_capacity(0, $SIZE); + match $VALUES { + None => (0..$SIZE).for_each(|_| { + builder.append_null(); + }), + Some(v) => { + let series = DataValue::try_into_data_array(&v, $D_TYPE)?; + (0..$SIZE).for_each(|_| { + builder.append_series(&series); + }) } } + Ok(builder.finish().into_series()) }}; } macro_rules! try_build_array { ($VALUE_BUILDER_TY:ident, $DF_TY:ident, $SCALAR_TY:ident, $VALUES:expr) => {{ - let mut builder = $VALUE_BUILDER_TY::::new($VALUES.len()); + let mut builder = $VALUE_BUILDER_TY::::with_capacity($VALUES.len()); for value in $VALUES.iter() { match value { DataValue::$SCALAR_TY(Some(v)) => builder.append_value(*v), @@ -172,7 +157,7 @@ macro_rules! try_build_array { // Boolean ($VALUES:expr) => {{ - let mut builder = BooleanArrayBuilder::new($VALUES.len()); + let mut builder = BooleanArrayBuilder::with_capacity($VALUES.len()); for value in $VALUES.iter() { match value { DataValue::Boolean(Some(v)) => builder.append_value(*v), @@ -185,7 +170,7 @@ macro_rules! try_build_array { // utf8 ($utf8:ident, $VALUES:expr) => {{ - let mut builder = Utf8ArrayBuilder::new($VALUES.len(), $VALUES.len() * 4); + let mut builder = Utf8ArrayBuilder::with_capacity($VALUES.len()); for value in $VALUES.iter() { match value { DataValue::Utf8(Some(v)) => builder.append_value(v), diff --git a/common/datavalues/src/prelude.rs b/common/datavalues/src/prelude.rs index c5096a907dfc0..22708218b2666 100644 --- a/common/datavalues/src/prelude.rs +++ b/common/datavalues/src/prelude.rs @@ -5,6 +5,7 @@ //! Everything you need to get started with this crate. pub use std::sync::Arc; +pub use crate::arrays::to_primitive; // arrays pub use crate::arrays::ArrayApply; pub use crate::arrays::ArrayApplyKernel; @@ -16,26 +17,24 @@ pub use crate::arrays::ArrayFull; pub use crate::arrays::ArrayFullNull; pub use crate::arrays::ArrayScatter; pub use crate::arrays::ArrayTake; -pub use crate::arrays::ArrayTakeEvery; pub use crate::arrays::BooleanArrayBuilder; pub use crate::arrays::DFUInt16ArrayBuilder; pub use crate::arrays::DFUInt32ArrayBuilder; pub use crate::arrays::DFUInt64ArrayBuilder; pub use crate::arrays::DFUInt8ArrayBuilder; pub use crate::arrays::DataArray; -pub use crate::arrays::GetValues; pub use crate::arrays::IntoTakeRandom; -pub use crate::arrays::IsNull; pub use crate::arrays::NewDataArray; pub use crate::arrays::PrimitiveArrayBuilder; pub use crate::arrays::TakeRandom; -pub use crate::arrays::ToPrimitive; pub use crate::arrays::Utf8ArrayBuilder; +pub use crate::bit_util::*; // columns pub use crate::columns::DataColumn; pub use crate::columns::DataColumnCommon; pub use crate::data_array_filter::*; pub use crate::data_df_type::*; +pub use crate::data_type::*; pub use crate::data_type_coercion::*; pub use crate::data_value::DFTryFrom; // series @@ -43,6 +42,7 @@ pub use crate::series::IntoSeries; pub use crate::series::Series; pub use crate::series::SeriesFrom; pub use crate::series::SeriesTrait; +pub use crate::utils::*; pub use crate::DFHasher; pub use crate::DFNumericType; pub use crate::DFPrimitiveType; @@ -52,7 +52,6 @@ pub use crate::DataGroupValue; pub use crate::DataSchema; pub use crate::DataSchemaRef; pub use crate::DataSchemaRefExt; -pub use crate::DataType; pub use crate::DataValue; pub use crate::DataValueAggregateOperator; pub use crate::DataValueAggregateOperator::*; @@ -63,3 +62,8 @@ pub use crate::DataValueComparisonOperator; pub use crate::DataValueComparisonOperator::*; pub use crate::DataValueLogicOperator; pub use crate::DataValueLogicOperator::*; + +pub type AlignedVec = common_arrow::arrow::buffer::MutableBuffer; +pub type LargeUtf8Array = common_arrow::arrow::array::Utf8Array; +pub type LargeBinaryArray = common_arrow::arrow::array::BinaryArray; +pub type LargeListArray = common_arrow::arrow::array::ListArray; diff --git a/common/datavalues/src/series/comparison.rs b/common/datavalues/src/series/comparison.rs index 1318c6cd339bd..43575a5ced18b 100644 --- a/common/datavalues/src/series/comparison.rs +++ b/common/datavalues/src/series/comparison.rs @@ -56,11 +56,6 @@ fn coerce_cmp_lhs_rhs(lhs: &Series, rhs: &Series) -> Result<(Series, Series)> { } impl ArrayCompare<&Series> for Series { - fn eq_missing(&self, rhs: &Series) -> Result { - let (lhs, rhs) = coerce_cmp_lhs_rhs(self, rhs)?; - impl_compare!(lhs.as_ref(), rhs.as_ref(), eq_missing) - } - /// Create a boolean mask by checking for equality. fn eq(&self, rhs: &Series) -> Result { let (lhs, rhs) = coerce_cmp_lhs_rhs(self, rhs)?; diff --git a/common/datavalues/src/series/date_wrap.rs b/common/datavalues/src/series/date_wrap.rs index 59442650749f8..aa7c216caa172 100644 --- a/common/datavalues/src/series/date_wrap.rs +++ b/common/datavalues/src/series/date_wrap.rs @@ -7,10 +7,10 @@ use std::fmt::Formatter; use std::sync::Arc; use common_arrow::arrow::array::ArrayRef; -use common_arrow::arrow::datatypes::IntervalUnit; use common_exception::Result; use crate::arrays::*; +use crate::prelude::*; use crate::series::wrap::SeriesWrap; use crate::series::*; use crate::*; diff --git a/common/datavalues/src/series/series_impl.rs b/common/datavalues/src/series/series_impl.rs index 1eab96d308236..4e95fa4cc2451 100644 --- a/common/datavalues/src/series/series_impl.rs +++ b/common/datavalues/src/series/series_impl.rs @@ -341,25 +341,6 @@ impl Series { } } - /// Check if all values in series are equal where `None == None` evaluates to `true`. - pub fn series_equal_missing(&self, other: &Series) -> bool { - if self.len() != other.len() { - return false; - } - if self.null_count() != other.null_count() { - return false; - } - // if all null and previous check did not return (so other is also all null) - if self.null_count() == self.len() { - return true; - } - - match self.eq_missing(other) { - Ok(arr) => arr.all_true(), - Err(_) => false, - } - } - /// Get a pointer to the underlying data of this Series. /// Can be useful for fast comparisons. pub fn get_data_ptr(&self) -> usize { diff --git a/common/datavalues/src/utils.rs b/common/datavalues/src/utils.rs index 51aaa3f326d41..feffd01d88201 100644 --- a/common/datavalues/src/utils.rs +++ b/common/datavalues/src/utils.rs @@ -4,6 +4,9 @@ use std::ops::Deref; use std::ops::DerefMut; +use common_arrow::arrow::bitmap::Bitmap; +use common_arrow::arrow::trusted_len::TrustedLen; + pub struct Wrap(pub T); impl Deref for Wrap { @@ -69,3 +72,84 @@ pub fn get_iter_capacity>(iter: &I) -> usize { (lower, None) => lower, } } + +pub struct TrustMyLength, J> { + iter: I, + len: usize, +} + +impl TrustMyLength +where I: Iterator +{ + #[inline] + pub fn new(iter: I, len: usize) -> Self { + Self { iter, len } + } +} + +impl Iterator for TrustMyLength +where I: Iterator +{ + type Item = J; + + #[inline] + fn next(&mut self) -> Option { + self.iter.next() + } + + fn size_hint(&self) -> (usize, Option) { + (self.len, Some(self.len)) + } +} + +impl ExactSizeIterator for TrustMyLength where I: Iterator {} + +impl DoubleEndedIterator for TrustMyLength +where I: Iterator + DoubleEndedIterator +{ + #[inline] + fn next_back(&mut self) -> Option { + self.iter.next_back() + } +} + +pub fn combine_validities(lhs: &Option, rhs: &Option) -> Option { + match (lhs, rhs) { + (Some(lhs), None) => Some(lhs.clone()), + (None, Some(rhs)) => Some(rhs.clone()), + (None, None) => None, + (Some(lhs), Some(rhs)) => Some(lhs & rhs), + } +} + +unsafe impl TrustedLen for TrustMyLength where I: Iterator {} + +pub trait CustomIterTools: Iterator { + fn fold_first_(mut self, f: F) -> Option + where + Self: Sized, + F: FnMut(Self::Item, Self::Item) -> Self::Item, + { + let first = self.next()?; + Some(self.fold(first, f)) + } + + fn trust_my_length(self, length: usize) -> TrustMyLength + where Self: Sized { + TrustMyLength::new(self, length) + } + + fn collect_trusted>(self) -> T + where Self: Sized + TrustedLen { + FromTrustedLenIterator::from_iter_trusted_length(self) + } +} + +pub trait CustomIterToolsSized: Iterator + Sized {} + +impl CustomIterTools for T where T: Iterator {} + +pub trait FromTrustedLenIterator: Sized { + fn from_iter_trusted_length>(iter: T) -> Self + where T::IntoIter: TrustedLen; +} diff --git a/common/datavalues/src/vec.rs b/common/datavalues/src/vec.rs deleted file mode 100644 index a7453e0fcb464..0000000000000 --- a/common/datavalues/src/vec.rs +++ /dev/null @@ -1,293 +0,0 @@ -// Copyright 2020-2021 The Datafuse Authors. -// -// SPDX-License-Identifier: Apache-2.0. - -use std::iter::FromIterator; -use std::mem; -use std::mem::ManuallyDrop; -use std::slice::IterMut; - -use common_arrow::arrow::alloc; -use common_arrow::arrow::array::ArrayData; -use common_arrow::arrow::array::PrimitiveArray; -use common_arrow::arrow::buffer::Buffer; -use common_arrow::arrow::buffer::MutableBuffer; -use common_arrow::arrow::datatypes::*; - -/// A `Vec` wrapper with a memory alignment equal to Arrow's primitive arrays. -/// Can be useful in creating a new DataArray or Arrow Primitive array without copying. -#[derive(Debug)] -pub struct AlignedVec { - pub inner: Vec, - // if into_inner is called, this will be true and we can use the default Vec's destructor - taken: bool, -} - -impl Drop for AlignedVec { - fn drop(&mut self) { - if !self.taken { - let inner = mem::take(&mut self.inner); - let mut me = mem::ManuallyDrop::new(inner); - let ptr: *mut T = me.as_mut_ptr(); - let ptr = ptr as *mut u8; - let ptr = std::ptr::NonNull::new(ptr).unwrap(); - unsafe { alloc::free_aligned::(ptr, me.capacity() * mem::size_of::()) } - } - } -} - -impl FromIterator for AlignedVec { - fn from_iter>(iter: I) -> Self { - let iter = iter.into_iter(); - let sh = iter.size_hint(); - let size = sh.1.unwrap_or(sh.0); - - let mut av = Self::with_capacity_aligned(size); - av.extend(iter); - - // Iterator size hint wasn't correct and reallocation has occurred - assert!(av.len() <= size); - av - } -} - -impl AlignedVec { - /// Uses a memcpy to initialize this AlignedVec - pub fn new_from_slice(other: &[T]) -> Self { - let len = other.len(); - let mut av = Self::with_capacity_aligned(len); - unsafe { - // Safety: - // we set initiate the memory after this with a memcpy. - av.set_len(len); - } - av.inner.copy_from_slice(other); - av - } -} - -impl AlignedVec { - pub fn resize(&mut self, new_len: usize, value: T) { - self.inner.resize(new_len, value) - } - - pub fn extend_from_slice(&mut self, other: &[T]) { - let remaining_cap = self.capacity() - self.len(); - let needed_cap = other.len(); - // exponential allocation - if needed_cap > remaining_cap { - self.reserve(std::cmp::max(needed_cap, self.capacity())); - } - self.inner.extend_from_slice(other) - } -} - -impl AlignedVec { - /// Create a new Vec where first bytes memory address has an alignment of 64 bytes, as described - /// by arrow spec. - /// Read more: - /// - pub fn with_capacity_aligned(size: usize) -> Self { - // Can only have a zero copy to arrow memory if address of first byte % 64 == 0 - let t_size = std::mem::size_of::(); - let capacity = size * t_size; - let ptr = alloc::allocate_aligned::(capacity).as_ptr() as *mut T; - let v = unsafe { Vec::from_raw_parts(ptr, 0, size) }; - AlignedVec { - inner: v, - taken: false, - } - } - - // with_capacity_aligned and set len = capacity - pub fn with_capacity_len_aligned(size: usize) -> Self { - let mut av = Self::with_capacity_aligned(size); - unsafe { - av.set_len(av.capacity()); - } - av - } - - pub fn is_empty(&self) -> bool { - self.inner.is_empty() - } - - #[inline] - pub fn reserve(&mut self, additional: usize) { - let mut me = ManuallyDrop::new(mem::take(&mut self.inner)); - let ptr = me.as_mut_ptr() as *mut u8; - let ptr = std::ptr::NonNull::new(ptr).unwrap(); - let t_size = mem::size_of::(); - let cap = me.capacity(); - let old_capacity = t_size * cap; - let new_capacity = old_capacity + t_size * additional; - let ptr = unsafe { alloc::reallocate::(ptr, old_capacity, new_capacity) }; - let ptr = ptr.as_ptr() as *mut T; - let v = unsafe { Vec::from_raw_parts(ptr, me.len(), cap + additional) }; - self.inner = v; - } - - #[inline] - pub fn len(&self) -> usize { - self.inner.len() - } - - /// Create a new aligned vec from a ptr. - /// - /// # Safety - /// The ptr should be 64 byte aligned and `len` and `capacity` should be correct otherwise it is UB. - pub unsafe fn from_ptr(ptr: usize, len: usize, capacity: usize) -> Self { - assert_eq!((ptr as usize) % alloc::ALIGNMENT, 0); - let ptr = ptr as *mut T; - let v = Vec::from_raw_parts(ptr, len, capacity); - Self { - inner: v, - taken: false, - } - } - - /// Take ownership of the Vec. This is UB because the destructor of Vec probably has a different - /// alignment than what we allocated. - /// - /// Only used for inner workings - unsafe fn into_inner(mut self) -> Vec { - if self.taken { - eprintln!("inner vec was already taken: UB"); - std::process::abort() - } - self.taken = true; - mem::take(&mut self.inner) - } - - pub fn iter_mut(&mut self) -> IterMut<'_, T> { - self.inner.iter_mut() - } - - /// Push at the end of the Vec. This is unsafe because a push when the capacity of the - /// inner Vec is reached will reallocate the Vec without the alignment, leaving this destructor's - /// alignment incorrect - #[inline] - pub fn push(&mut self, value: T) { - if self.inner.len() == self.capacity() { - // exponential allocation - self.reserve(std::cmp::max(self.capacity(), 5)); - } - self.inner.push(value) - } - - /// Set the length of the underlying `Vec`. - /// - /// # Safety - /// - /// - `new_len` must be less than or equal to `capacity`. - /// - The elements at `old_len..new_len` must be initialized. - #[inline] - pub unsafe fn set_len(&mut self, new_len: usize) { - self.inner.set_len(new_len); - } - - #[inline] - pub fn as_ptr(&self) -> *const T { - self.inner.as_ptr() - } - - #[inline] - pub fn as_mut_ptr(&mut self) -> *mut T { - self.inner.as_mut_ptr() - } - - #[inline] - pub fn as_mut_slice(&mut self) -> &mut [T] { - self.inner.as_mut_slice() - } - - #[inline] - pub fn capacity(&self) -> usize { - self.inner.capacity() - } - - pub fn shrink_to_fit(&mut self) { - if self.capacity() > self.len() && !self.is_empty() { - let mut me = ManuallyDrop::new(mem::take(&mut self.inner)); - let ptr = me.as_mut_ptr() as *mut u8; - let ptr = std::ptr::NonNull::new(ptr).unwrap(); - - let t_size = mem::size_of::(); - let new_size = t_size * me.len(); - let old_size = t_size * me.capacity(); - let v = unsafe { - let ptr = alloc::reallocate::(ptr, old_size, new_size).as_ptr() as *mut T; - Vec::from_raw_parts(ptr, me.len(), me.len()) - }; - - self.inner = v; - } - } - - /// Transform this array to an Arrow Buffer. - pub fn into_arrow_buffer(self) -> Buffer { - if self.is_empty() && self.capacity() == 0 { - MutableBuffer::new(0).into() - } else { - let values = unsafe { self.into_inner() }; - - let me = mem::ManuallyDrop::new(values); - let ptr = me.as_ptr() as *mut u8; - let len = me.len() * std::mem::size_of::(); - let capacity = me.capacity() * std::mem::size_of::(); - debug_assert_eq!((ptr as usize) % 64, 0); - let ptr = std::ptr::NonNull::new(ptr).unwrap(); - - unsafe { Buffer::from_raw_parts(ptr, len, capacity) } - } - } - - pub fn into_primitive_array( - self, - null_buf: Option, - ) -> PrimitiveArray { - debug_assert_eq!(mem::size_of::(), mem::size_of::()); - - let vec_len = self.len(); - let buffer = self.into_arrow_buffer(); - - let mut builder = ArrayData::builder(A::DATA_TYPE) - .len(vec_len) - .add_buffer(buffer); - - if let Some(buf) = null_buf { - builder = builder.null_bit_buffer(buf); - } - let data = builder.build(); - - PrimitiveArray::::from(data) - } - - /// # Panic - /// Must be a trusted len iterator or else it will panic - pub fn extend>(&mut self, iter: I) { - let iter = iter.into_iter(); - let cap = iter.size_hint().1.expect("a trusted length iterator"); - let (extra_cap, overflow) = cap.overflowing_sub(self.capacity()); - - if extra_cap > 0 && !overflow { - self.reserve(extra_cap); - } - let len_before = self.len(); - self.inner.extend(iter); - let added = self.len() - len_before; - - if added != cap { - eprintln!("size hint was incorrect, this is UB. aborting"); - std::process::abort() - } - } -} - -impl Default for AlignedVec { - fn default() -> Self { - // Be careful here. Don't initialize with a normal Vec as this will cause the wrong deallocator - // to run and SIGSEGV - Self::with_capacity_aligned(0) - } -} diff --git a/common/datavalues/src/vec_test.rs b/common/datavalues/src/vec_test.rs deleted file mode 100644 index 43ccbd2348345..0000000000000 --- a/common/datavalues/src/vec_test.rs +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2020-2021 The Datafuse Authors. -// -// SPDX-License-Identifier: Apache-2.0. - -#[test] -fn test_aligned_vec_allocations() { - use common_arrow::arrow::alloc; - - use super::*; - - // Can only have a zero copy to arrow memory if address of first byte % 64 == 0 - // check if we can increase above initial capacity and keep the Arrow alignment - let mut v = AlignedVec::with_capacity_aligned(2); - v.push(1); - v.push(2); - v.push(3); - v.push(4); - - let ptr = v.as_ptr(); - assert_eq!((ptr as usize) % alloc::ALIGNMENT, 0); - - // check if we can shrink to fit - let mut v = AlignedVec::with_capacity_aligned(10); - v.push(1); - v.push(2); - v.shrink_to_fit(); - assert_eq!(v.len(), 2); - assert_eq!(v.capacity(), 2); - let ptr = v.as_ptr(); - assert_eq!((ptr as usize) % alloc::ALIGNMENT, 0); - - let a = v.into_primitive_array::(None); - assert_eq!(&a.values()[..2], &[1, 2]) -} diff --git a/common/flights/src/impls/storage_api_impl.rs b/common/flights/src/impls/storage_api_impl.rs index 9dabc11ea0e23..2df2122b40fe4 100644 --- a/common/flights/src/impls/storage_api_impl.rs +++ b/common/flights/src/impls/storage_api_impl.rs @@ -5,12 +5,13 @@ use std::convert::TryFrom; +// io::ipc::write::common::{encoded_batch, DictionaryTracker, EncodedData, IpcWriteOptions} use common_arrow::arrow::datatypes::SchemaRef as ArrowSchemaRef; -use common_arrow::arrow::ipc::writer::IpcWriteOptions; +use common_arrow::arrow::io::ipc::write::common::IpcWriteOptions; use common_arrow::arrow::record_batch::RecordBatch; use common_arrow::arrow_flight::utils::flight_data_from_arrow_batch; +use common_arrow::arrow_flight::utils::flight_data_from_arrow_schema; use common_arrow::arrow_flight::utils::flight_data_to_arrow_batch; -use common_arrow::arrow_flight::SchemaAsIpc; use common_arrow::arrow_flight::Ticket; use common_datablocks::DataBlock; use common_datavalues::prelude::*; @@ -81,7 +82,7 @@ impl StorageApi for StoreClient { let res_stream = res.map(move |item| { item.map_err(|status| ErrorCode::TokioError(status.to_string())) .and_then(|item| { - flight_data_to_arrow_batch(&item, arrow_schema.clone(), &[]) + flight_data_to_arrow_batch(&item, arrow_schema.clone(), true, &[]) .map_err(ErrorCode::from) }) .and_then(DataBlock::try_from) @@ -98,7 +99,8 @@ impl StorageApi for StoreClient { ) -> common_exception::Result { let ipc_write_opt = IpcWriteOptions::default(); let arrow_schema: ArrowSchemaRef = Arc::new(scheme_ref.to_arrow()); - let flight_schema = SchemaAsIpc::new(arrow_schema.as_ref(), &ipc_write_opt).into(); + + let flight_schema = flight_data_from_arrow_schema(arrow_schema.as_ref(), &ipc_write_opt); let (mut tx, flight_stream) = futures::channel::mpsc::channel(100); tx.send(flight_schema) .await diff --git a/common/functions/src/aggregates/aggregate_combinator_if.rs b/common/functions/src/aggregates/aggregate_combinator_if.rs index 073c37ace2ff8..2617cafed5787 100644 --- a/common/functions/src/aggregates/aggregate_combinator_if.rs +++ b/common/functions/src/aggregates/aggregate_combinator_if.rs @@ -6,6 +6,7 @@ use std::fmt; use bytes::BytesMut; use common_arrow::arrow; +use common_arrow::arrow::array::*; use common_datavalues::prelude::*; use common_exception::ErrorCode; use common_exception::Result; @@ -92,6 +93,7 @@ impl AggregateFunction for AggregateIfCombinator { let boolean_array = boolean_array.bool()?; let arrow_filter_array = boolean_array.downcast_ref(); + let bitmap = arrow_filter_array.values(); let mut column_array = Vec::with_capacity(self.argument_len - 1); let row_size = match columns.len() - 1 { @@ -100,18 +102,24 @@ impl AggregateFunction for AggregateIfCombinator { if boolean_array.null_count() > 0 { // this greatly simplifies subsequent filtering code // now we only have a boolean mask to deal with - arrow::compute::prep_null_mask_filter(arrow_filter_array) - .values() - .count_set_bits() + let boolean_bm = arrow_filter_array.validity(); + let res = combine_validities(&Some(bitmap.clone()), boolean_bm); + match res { + Some(v) => v.len() - v.null_count(), + None => 0, + } } else { - arrow_filter_array.values().count_set_bits() + bitmap.len() - bitmap.null_count() } } 1 => { // single array handle let array = columns[0].to_array()?; - let data = - arrow::compute::filter(array.get_array_ref().as_ref(), arrow_filter_array)?; + let data = arrow::compute::filter::filter( + array.get_array_ref().as_ref(), + arrow_filter_array, + )?; + let data: ArrayRef = Arc::from(data); column_array.push(DataColumn::from(data)); column_array[0].len() } diff --git a/common/functions/src/aggregates/aggregate_min_max.rs b/common/functions/src/aggregates/aggregate_min_max.rs index 14c777c2f7da5..effc59a435c4d 100644 --- a/common/functions/src/aggregates/aggregate_min_max.rs +++ b/common/functions/src/aggregates/aggregate_min_max.rs @@ -144,6 +144,7 @@ where } } +#[inline] pub fn min_batch(column: &DataColumn) -> Result { if column.is_empty() { return Ok(DataValue::from(&column.data_type())); @@ -154,6 +155,7 @@ pub fn min_batch(column: &DataColumn) -> Result { } } +#[inline] pub fn max_batch(column: &DataColumn) -> Result { if column.is_empty() { return Ok(DataValue::from(&column.data_type())); diff --git a/common/functions/src/aggregates/aggregate_sum.rs b/common/functions/src/aggregates/aggregate_sum.rs index 839991b0c1886..e3f4426d23249 100644 --- a/common/functions/src/aggregates/aggregate_sum.rs +++ b/common/functions/src/aggregates/aggregate_sum.rs @@ -180,6 +180,7 @@ where } } +#[inline] pub fn sum_batch(column: &DataColumn) -> Result { if column.is_empty() { return Ok(DataValue::Null); diff --git a/common/functions/src/scalars/strings/substring.rs b/common/functions/src/scalars/strings/substring.rs index b227e4edf4cc1..001c184587cb5 100644 --- a/common/functions/src/scalars/strings/substring.rs +++ b/common/functions/src/scalars/strings/substring.rs @@ -54,8 +54,8 @@ impl Function for SubstringFunction { // todo, move these to datavalues let value = columns[0].to_array()?; let arrow_array = value.get_array_ref(); - let result = - compute::kernels::substring::substring(arrow_array.as_ref(), from, &end)? as ArrayRef; + let result = compute::substring::substring(arrow_array.as_ref(), from, &end)?; + let result: ArrayRef = Arc::from(result); Ok(result.into()) } diff --git a/common/streams/src/stream_limit_by.rs b/common/streams/src/stream_limit_by.rs index 3a74e2bbad3d1..2980e596199d7 100644 --- a/common/streams/src/stream_limit_by.rs +++ b/common/streams/src/stream_limit_by.rs @@ -8,10 +8,11 @@ use std::task::Context; use std::task::Poll; use common_arrow::arrow; -use common_arrow::arrow::array::BooleanArray; use common_datablocks::DataBlock; use common_datablocks::HashMethod; use common_datablocks::HashMethodSerializer; +use common_datavalues::prelude::*; +use common_datavalues::DFBooleanArray; use common_exception::Result; use futures::Stream; use futures::StreamExt; @@ -53,9 +54,11 @@ impl LimitByStream { } } - let filter_array = BooleanArray::from(filter_vec); + let filter_array = DFBooleanArray::new_from_slice(&filter_vec); + let batch = block.clone().try_into()?; - let batch = arrow::compute::filter_record_batch(&batch, &filter_array)?; + let batch = + arrow::compute::filter::filter_record_batch(&batch, filter_array.downcast_ref())?; Some(batch.try_into()).transpose() } } diff --git a/fusequery/query/src/api/rpc/flight_client_stream.rs b/fusequery/query/src/api/rpc/flight_client_stream.rs index 31059942ebfc4..acf5b51ac4581 100644 --- a/fusequery/query/src/api/rpc/flight_client_stream.rs +++ b/fusequery/query/src/api/rpc/flight_client_stream.rs @@ -37,14 +37,16 @@ impl FlightDataStream { .collect::>(); DataBlock::create( - Arc::new(DataSchema::from(record_batch.schema())), + Arc::new(DataSchema::from(record_batch.schema().as_ref())), columns, ) } let arrow_schema = Arc::new(schema.to_arrow()); - Ok(flight_data_to_arrow_batch(&flight_data, arrow_schema, &[]) - .map(create_data_block)?) + Ok( + flight_data_to_arrow_batch(&flight_data, arrow_schema, true, &[]) + .map(create_data_block)?, + ) } } }) @@ -67,14 +69,17 @@ impl FlightDataStream { .map(|column| DataColumn::Array(column.clone().into_series())) .collect::>(); - let schema = DataSchema::from(record_batch.schema()); + let schema = DataSchema::from(record_batch.schema().as_ref()); DataBlock::create(Arc::new(schema), columns) } - Ok( - flight_data_to_arrow_batch(&flight_data, Arc::new(schema_ref.to_arrow()), &[]) - .map(create_data_block)?, + Ok(flight_data_to_arrow_batch( + &flight_data, + Arc::new(schema_ref.to_arrow()), + true, + &[], ) + .map(create_data_block)?) } }) } diff --git a/fusequery/query/src/api/rpc/flight_service_stream.rs b/fusequery/query/src/api/rpc/flight_service_stream.rs index 232b831e36c21..05533f28ddecc 100644 --- a/fusequery/query/src/api/rpc/flight_service_stream.rs +++ b/fusequery/query/src/api/rpc/flight_service_stream.rs @@ -4,7 +4,7 @@ use std::convert::TryInto; -use common_arrow::arrow::ipc::writer::IpcWriteOptions; +use common_arrow::arrow::io::ipc::write::common::IpcWriteOptions; use common_arrow::arrow_flight::utils::flight_data_from_arrow_batch; use common_arrow::arrow_flight::FlightData; use common_datablocks::DataBlock; diff --git a/fusequery/query/src/datasources/local/csv_table_stream.rs b/fusequery/query/src/datasources/local/csv_table_stream.rs index 62ad9428edd8a..a9750ded6f06b 100644 --- a/fusequery/query/src/datasources/local/csv_table_stream.rs +++ b/fusequery/query/src/datasources/local/csv_table_stream.rs @@ -2,12 +2,11 @@ // // SPDX-License-Identifier: Apache-2.0. -use std::convert::TryInto; -use std::fs::File; +use std::convert::TryFrom; use std::sync::Arc; use std::task::Poll; -use common_arrow::arrow::csv; +use common_arrow::arrow::io::csv::read; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; @@ -41,23 +40,28 @@ impl CsvTableStream { let names: Vec<_> = part.name.split('-').collect(); let begin: usize = names[1].parse()?; let end: usize = names[2].parse()?; - let bounds = Some((begin, end)); let block_size = end - begin; - let file = File::open(self.file.clone())?; let arrow_schema = Arc::new(self.schema.to_arrow()); - let mut reader: csv::Reader = - csv::Reader::new(file, arrow_schema, false, None, block_size, bounds, None); + let mut reader = read::ReaderBuilder::new() + .has_headers(false) + .from_path(&self.file) + .map_err(|e| ErrorCode::CannotReadFile(e.to_string()))?; - reader - .next() - .map(|record| { - record - .map_err(ErrorCode::from) - .and_then(|record| record.try_into()) - }) - .map(|data_block| data_block.map(Some)) - .unwrap_or_else(|| Ok(None)) + let mut rows = vec![read::ByteRecord::default(); block_size]; + let rows_read = read::read_rows(&mut reader, begin, &mut rows)?; + let rows = &rows[..rows_read]; + + let record = read::deserialize_batch( + rows, + arrow_schema.fields(), + None, + 0, + read::deserialize_column, + )?; + + let block = DataBlock::try_from(record)?; + Ok(Some(block)) } } diff --git a/fusequery/query/src/datasources/local/csv_table_test.rs b/fusequery/query/src/datasources/local/csv_table_test.rs index 11e91f84204ab..73f6281ec02c0 100644 --- a/fusequery/query/src/datasources/local/csv_table_test.rs +++ b/fusequery/query/src/datasources/local/csv_table_test.rs @@ -4,6 +4,7 @@ use std::env; +use common_datablocks::assert_blocks_sorted_eq; use common_datavalues::prelude::*; use common_exception::Result; use common_planners::*; @@ -126,13 +127,22 @@ async fn test_csv_table_parse_error() -> Result<()> { let stream = table.read(ctx, &source_plan).await?; let result = stream.try_collect::>().await; - assert_eq!(true, result.is_err()); - if let Err(e) = result { - assert_eq!( - "Code: 1002, displayText = Parser error: Error while parsing value \'Shanghai\' for column 1 at line 1.", - e.to_string() - ); - }; - + // integer parse error will result to Null value + assert_eq!(false, result.is_err()); + assert_blocks_sorted_eq( + vec![ + "+---------+---------+---------+---------+", + "| column1 | column2 | column3 | column4 |", + "+---------+---------+---------+---------+", + "| 1 | NULL | 100 | NULL |", + "| 2 | NULL | 80 | NULL |", + "| 3 | NULL | 60 | NULL |", + "| 4 | NULL | 70 | NULL |", + "| 5 | NULL | 55 | NULL |", + "| 6 | NULL | 99 | NULL |", + "+---------+---------+---------+---------+", + ], + &result.unwrap(), + ); Ok(()) } diff --git a/fusequery/query/src/datasources/local/parquet_table.rs b/fusequery/query/src/datasources/local/parquet_table.rs index 3727006de6303..ddcaf4f1227d1 100644 --- a/fusequery/query/src/datasources/local/parquet_table.rs +++ b/fusequery/query/src/datasources/local/parquet_table.rs @@ -7,9 +7,7 @@ use std::convert::TryInto; use std::fs::File; use std::sync::Arc; -use common_arrow::parquet::arrow::ArrowReader; -use common_arrow::parquet::arrow::ParquetFileArrowReader; -use common_arrow::parquet::file::reader::SerializedFileReader; +use common_arrow::arrow::io::parquet::read; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; @@ -66,27 +64,21 @@ fn read_file( tx: Sender>>, projection: &[usize], ) -> Result<()> { - let file_reader = File::open(file).map_err(|e| ErrorCode::CannotReadFile(e.to_string()))?; - let file_reader = SerializedFileReader::new(file_reader) - .map_err(|e| ErrorCode::ParquetError(e.to_string()))?; - let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader)); - - // TODO projection, row filters, batch size configurable, schema judgement - let batch_size = 2048; - let mut batch_reader = arrow_reader - .get_record_reader_by_columns(projection.to_owned(), batch_size) - .map_err(|exception| ErrorCode::ParquetError(exception.to_string()))?; - - loop { - match batch_reader.next() { - Some(Ok(batch)) => { + let reader = File::open(file)?; + let reader = read::RecordReader::try_new( + reader, + Some(projection.to_vec()), + None, + Arc::new(|_, _| true), + )?; + + for maybe_batch in reader { + match maybe_batch { + Ok(batch) => { tx.send(Some(Ok(batch.try_into()?))) .map_err(|e| ErrorCode::UnknownException(e.to_string()))?; } - None => { - break; - } - Some(Err(e)) => { + Err(e) => { let err_msg = format!("Error reading batch from {:?}: {}", file, e.to_string()); tx.send(Some(Result::Err(ErrorCode::CannotReadFile( @@ -98,6 +90,7 @@ fn read_file( } } } + Ok(()) } diff --git a/fusequery/query/src/datasources/system/numbers_stream.rs b/fusequery/query/src/datasources/system/numbers_stream.rs index 488e91755dbe1..20ca3f193fd06 100644 --- a/fusequery/query/src/datasources/system/numbers_stream.rs +++ b/fusequery/query/src/datasources/system/numbers_stream.rs @@ -8,7 +8,6 @@ use std::usize; use common_datablocks::DataBlock; use common_datavalues::prelude::*; -use common_datavalues::AlignedVec; use common_datavalues::DFUInt64Array; use common_exception::Result; use common_streams::ProgressStream; @@ -40,6 +39,7 @@ impl NumbersStream { ProgressStream::try_create(stream, ctx.progress_callback()?) } + #[inline] fn try_get_one_block(&mut self) -> Result> { if (self.block_index as usize) == self.blocks.len() { let partitions = self.ctx.try_get_partitions(1)?; @@ -87,12 +87,18 @@ impl NumbersStream { Ok(if current.begin == current.end { None } else { - let mut av = - AlignedVec::with_capacity_len_aligned((current.end - current.begin) as usize); + let size = (current.end - current.begin) as usize; + let mut av = AlignedVec::with_capacity(size); + + unsafe { av.set_len(size) }; + + av.as_mut_slice() + .iter_mut() + .enumerate() + .for_each(|(idx, num)| { + *num = current.begin + idx as u64; + }); - av.iter_mut().enumerate().for_each(|(idx, num)| { - *num = current.begin + idx as u64; - }); let series = DFUInt64Array::new_from_aligned_vec(av).into_series(); let block = DataBlock::create_by_array(self.schema.clone(), vec![series]); Some(block) diff --git a/fusequery/query/src/pipelines/transforms/transform_aggregator_partial.rs b/fusequery/query/src/pipelines/transforms/transform_aggregator_partial.rs index 4538697b4d755..81cbb21a27a5c 100644 --- a/fusequery/query/src/pipelines/transforms/transform_aggregator_partial.rs +++ b/fusequery/query/src/pipelines/transforms/transform_aggregator_partial.rs @@ -108,7 +108,7 @@ impl Processor for AggregatorPartialTransform { let mut bytes = BytesMut::new(); for (idx, func) in funcs.iter().enumerate() { func.serialize(places[idx], &mut bytes)?; - let mut array_builder = BinaryArrayBuilder::new(4); + let mut array_builder = BinaryArrayBuilder::with_capacity(4); array_builder.append_value(&bytes[..]); bytes.clear(); let array = array_builder.finish(); diff --git a/fusequery/query/src/pipelines/transforms/transform_filter.rs b/fusequery/query/src/pipelines/transforms/transform_filter.rs index f03eebe988518..6e9879ef2c1a2 100644 --- a/fusequery/query/src/pipelines/transforms/transform_filter.rs +++ b/fusequery/query/src/pipelines/transforms/transform_filter.rs @@ -94,7 +94,7 @@ impl Processor for FilterTransform { let filter_array = filter_array.bool()?.downcast_ref(); // Convert to arrow record_batch let batch = block.try_into()?; - let batch = arrow::compute::filter_record_batch(&batch, filter_array)?; + let batch = arrow::compute::filter::filter_record_batch(&batch, filter_array)?; let delta = start.elapsed(); tracing::debug!("Filter cost: {:?}", delta); diff --git a/fusequery/query/src/pipelines/transforms/transform_group_by_partial.rs b/fusequery/query/src/pipelines/transforms/transform_group_by_partial.rs index 4f7122034832c..9f202bf15c755 100644 --- a/fusequery/query/src/pipelines/transforms/transform_group_by_partial.rs +++ b/fusequery/query/src/pipelines/transforms/transform_group_by_partial.rs @@ -202,11 +202,11 @@ impl Processor for GroupByPartialTransform { // Builders. let mut state_builders: Vec = (0..aggr_len) - .map(|_| BinaryArrayBuilder::new(groups.len() * 4)) + .map(|_| BinaryArrayBuilder::with_capacity(groups.len() * 4)) .collect(); type KeyBuilder = $key_array_builder; - let mut group_key_builder = KeyBuilder::new(groups.len()); + let mut group_key_builder = KeyBuilder::with_capacity(groups.len()); let mut bytes = BytesMut::new(); for (key, (places, values)) in groups.iter() { diff --git a/fusequery/query/src/servers/clickhouse/writers/query_writer.rs b/fusequery/query/src/servers/clickhouse/writers/query_writer.rs index 36c4eb6366425..f88accc912b7a 100644 --- a/fusequery/query/src/servers/clickhouse/writers/query_writer.rs +++ b/fusequery/query/src/servers/clickhouse/writers/query_writer.rs @@ -184,42 +184,54 @@ pub fn to_clickhouse_block(block: DataBlock) -> Result { } }, false => match column.data_type() { - DataType::Int8 => { - result.column(name, column.i8()?.downcast_ref().values().to_owned()) - } - DataType::Int16 => { - result.column(name, column.i16()?.downcast_ref().values().to_owned()) - } - DataType::Int32 => { - result.column(name, column.i32()?.downcast_ref().values().to_owned()) - } - DataType::Int64 => { - result.column(name, column.i64()?.downcast_ref().values().to_owned()) - } - DataType::UInt8 => { - result.column(name, column.u8()?.downcast_ref().values().to_owned()) - } - DataType::UInt16 => { - result.column(name, column.u16()?.downcast_ref().values().to_owned()) - } - DataType::UInt32 => { - result.column(name, column.u32()?.downcast_ref().values().to_owned()) - } - DataType::UInt64 => { - result.column(name, column.u64()?.downcast_ref().values().to_owned()) - } - DataType::Float32 => { - result.column(name, column.f32()?.downcast_ref().values().to_owned()) - } - DataType::Float64 => { - result.column(name, column.f64()?.downcast_ref().values().to_owned()) - } - DataType::Date32 => { - result.column(name, column.date32()?.downcast_ref().values().to_owned()) - } - DataType::Date64 => { - result.column(name, column.date64()?.downcast_ref().values().to_owned()) - } + DataType::Int8 => result.column( + name, + column.i8()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Int16 => result.column( + name, + column.i16()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Int32 => result.column( + name, + column.i32()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Int64 => result.column( + name, + column.i64()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::UInt8 => result.column( + name, + column.u8()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::UInt16 => result.column( + name, + column.u16()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::UInt32 => result.column( + name, + column.u32()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::UInt64 => result.column( + name, + column.u64()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Float32 => result.column( + name, + column.f32()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Float64 => result.column( + name, + column.f64()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Date32 => result.column( + name, + column.date32()?.downcast_ref().values().as_slice().to_vec(), + ), + DataType::Date64 => result.column( + name, + column.date64()?.downcast_ref().values().as_slice().to_vec(), + ), DataType::Utf8 => { let vs: Vec<&str> = column.utf8()?.downcast_iter().map(|c| c.unwrap()).collect(); diff --git a/fusequery/query/src/sql/sql_common.rs b/fusequery/query/src/sql/sql_common.rs index f270ba5c4e01b..7586b65f1c02b 100644 --- a/fusequery/query/src/sql/sql_common.rs +++ b/fusequery/query/src/sql/sql_common.rs @@ -4,9 +4,7 @@ use std::str::FromStr; -use common_arrow::arrow::datatypes::TimeUnit; -use common_datavalues::DataType; -use common_datavalues::DataValue; +use common_datavalues::prelude::*; use common_exception::ErrorCode; use common_exception::Result; use common_planners::Expression; diff --git a/fusestore/store/src/data_part/appender.rs b/fusestore/store/src/data_part/appender.rs index 8ff4a6862c1c5..3235d4731602b 100644 --- a/fusestore/store/src/data_part/appender.rs +++ b/fusestore/store/src/data_part/appender.rs @@ -4,16 +4,18 @@ // use std::convert::TryFrom; +use std::io::Cursor; +use std::iter::repeat; use std::sync::Arc; -use anyhow::Context; use anyhow::Result; use common_arrow::arrow::datatypes::Schema as ArrowSchema; +use common_arrow::arrow::io::parquet::write::*; use common_arrow::arrow::record_batch::RecordBatch; use common_arrow::arrow_flight::utils::flight_data_to_arrow_batch; use common_arrow::arrow_flight::FlightData; -use common_arrow::parquet::arrow::ArrowWriter; -use common_arrow::parquet::file::writer::InMemoryWriteableCursor; +// use common_arrow::parquet::arrow::ArrowWriter; +// use common_arrow::parquet::file::writer::InMemoryWriteableCursor; use common_datablocks::DataBlock; use common_flights::storage_api_impl::AppendResult; use futures::StreamExt; @@ -43,7 +45,7 @@ impl Appender { let mut result = AppendResult::default(); while let Some(flight_data) = stream.next().await { let batch = - flight_data_to_arrow_batch(&flight_data, arrow_schema_ref.clone(), &[])?; + flight_data_to_arrow_batch(&flight_data, arrow_schema_ref.clone(), true, &[])?; let block = DataBlock::try_from(batch)?; let (rows, cols, wire_bytes) = (block.num_rows(), block.num_columns(), block.memory_size()); @@ -63,15 +65,34 @@ impl Appender { } pub(crate) fn write_in_memory(block: DataBlock) -> Result> { - let cursor = InMemoryWriteableCursor::default(); - { - let cursor = cursor.clone(); - let batch = RecordBatch::try_from(block)?; - let mut writer = ArrowWriter::try_new(cursor, batch.schema(), None)?; - writer.write(&batch)?; - writer.close()?; - } - cursor - .into_inner() - .context("failed to convert cursor into vector of u8") + let arrow_schema = block.schema().to_arrow(); + let options = WriteOptions { + write_statistics: true, + compression: CompressionCodec::Uncompressed, + version: Version::V2, + }; + let encodings: Vec<_> = repeat(Encoding::Plain).take(block.num_columns()).collect(); + let memory_size = block.memory_size(); + let batch = RecordBatch::try_from(block)?; + + let iter = vec![Ok(batch)]; + + let row_groups = + RowGroupIterator::try_new(iter.into_iter(), &arrow_schema, options, encodings)?; + + // Create a new empty file + let writer = Vec::with_capacity(memory_size); + let mut cursor = Cursor::new(writer); + // Write the file. Note that, at present, any error results in a corrupted file. + let parquet_schema = row_groups.parquet_schema().clone(); + write_file( + &mut cursor, + row_groups, + &arrow_schema, + parquet_schema, + options, + None, + )?; + + Ok(cursor.into_inner()) } diff --git a/fusestore/store/src/data_part/appender_test.rs b/fusestore/store/src/data_part/appender_test.rs index 76fdcc0975f87..3ddce8d42616e 100644 --- a/fusestore/store/src/data_part/appender_test.rs +++ b/fusestore/store/src/data_part/appender_test.rs @@ -5,20 +5,17 @@ #[cfg(test)] mod test { + use std::io::Cursor; use std::sync::Arc; use anyhow::bail; use common_arrow::arrow::array::ArrayRef; use common_arrow::arrow::array::Int64Array; - use common_arrow::arrow::array::StringArray; - use common_arrow::arrow::ipc::writer::IpcWriteOptions; + use common_arrow::arrow::io::ipc::write::common::IpcWriteOptions; + use common_arrow::arrow::io::parquet::read; use common_arrow::arrow::record_batch::RecordBatch; use common_arrow::arrow_flight::utils::flight_data_from_arrow_batch; - use common_arrow::arrow_flight::SchemaAsIpc; - use common_arrow::parquet::arrow::ArrowReader; - use common_arrow::parquet::arrow::ParquetFileArrowReader; - use common_arrow::parquet::file::reader::SerializedFileReader; - use common_arrow::parquet::file::serialized_reader::SliceableCursor; + use common_arrow::arrow_flight::utils::flight_data_from_arrow_schema; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_runtime::tokio; @@ -38,16 +35,12 @@ mod test { let block = DataBlock::create_by_array(schema.clone(), vec![col0.clone(), col1.clone()]); let buffer = write_in_memory(block)?; + let cursor = Cursor::new(buffer); + let mut reader = read::RecordReader::try_new(cursor, None, None, Arc::new(|_, _| true))?; + let arrow_schema = schema.to_arrow(); - let cursor = SliceableCursor::new(buffer); - let reader = SerializedFileReader::new(cursor)?; - let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(reader)); - let arrow_schema = arrow_reader.get_schema()?; - assert_eq!(arrow_schema, schema.to_arrow()); - - let mut records = arrow_reader.get_record_reader(1024)?; - if let Some(r) = records.next() { - let batch = r?; + if let Some(maybe_batch) = reader.next() { + let batch = maybe_batch?; assert_eq!(batch.schema().as_ref(), &arrow_schema); assert_eq!(batch.column(0), &col0.get_array_ref()); assert_eq!(batch.column(1), &col1.get_array_ref()); @@ -59,8 +52,10 @@ mod test { #[tokio::test(flavor = "multi_thread", worker_threads = 1)] async fn test_append() -> anyhow::Result<()> { - let col0: ArrayRef = Arc::new(Int64Array::from(vec![0, 1, 2])); - let col1: ArrayRef = Arc::new(StringArray::from(vec!["str1", "str2", "str3"])); + let col0: ArrayRef = Arc::new(Int64Array::from_values(vec![0, 1, 2])); + let col1: ArrayRef = Arc::new(LargeUtf8Array::from_iter_values( + vec!["str1", "str2", "str3"].iter(), + )); let batch = RecordBatch::try_from_iter(vec![("col0", col0), ("col1", col1)])?; let schema = batch.schema(); @@ -71,7 +66,7 @@ mod test { let appender = Appender::new(Arc::new(fs)); let default_ipc_write_opt = IpcWriteOptions::default(); - let flight_schema = SchemaAsIpc::new(&schema, &default_ipc_write_opt).into(); + let flight_schema = flight_data_from_arrow_schema(&schema, &default_ipc_write_opt); let req = futures::stream::iter(vec![ flight_schema, diff --git a/fusestore/store/src/executor/action_handler.rs b/fusestore/store/src/executor/action_handler.rs index 66d72fd6a8c94..5a7a523748905 100644 --- a/fusestore/store/src/executor/action_handler.rs +++ b/fusestore/store/src/executor/action_handler.rs @@ -2,16 +2,14 @@ // // SPDX-License-Identifier: Apache-2.0. +use std::io::Cursor; use std::pin::Pin; use std::sync::Arc; -use common_arrow::arrow::ipc::writer::IpcWriteOptions; +use common_arrow::arrow::io::ipc::write::common::IpcWriteOptions; +use common_arrow::arrow::io::parquet::read; use common_arrow::arrow_flight::utils::flight_data_from_arrow_batch; use common_arrow::arrow_flight::FlightData; -use common_arrow::parquet::arrow::ArrowReader; -use common_arrow::parquet::arrow::ParquetFileArrowReader; -use common_arrow::parquet::file::reader::SerializedFileReader; -use common_arrow::parquet::file::serialized_reader::SliceableCursor; use common_exception::ErrorCode; use common_flights::storage_api_impl::AppendResult; use common_flights::storage_api_impl::ReadAction; @@ -170,29 +168,26 @@ impl ActionHandler { return Err(ErrorCode::IllegalScanPlan("invalid PlanNode passed in")); }; - let content = self.fs.read_all(&part_file).await?; - let cursor = SliceableCursor::new(content); - - let file_reader = SerializedFileReader::new(cursor) - .map_err(|pe| ErrorCode::ReadFileError(format!("parquet error: {}", pe.to_string())))?; - let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader)); - // before push_down is passed in, we returns all the columns let schema = plan.schema; let projection = (0..schema.fields().len()).collect::>(); - // TODO config - let batch_size = 2048; + // TODO expose a reader from fs + let content = self.fs.read_all(&part_file).await?; + let reader = Cursor::new(content); - let batch_reader = arrow_reader - .get_record_reader_by_columns(projection, batch_size) - .map_err(|pe| ErrorCode::ReadFileError(format!("parquet error: {}", pe.to_string())))?; + let reader = read::RecordReader::try_new( + reader, + Some(projection.to_vec()), + None, + Arc::new(|_, _| true), + )?; // For simplicity, we do the conversion in-memory, to be optimized later // TODO consider using `parquet_table` and `stream_parquet` let write_opt = IpcWriteOptions::default(); let flights = - batch_reader + reader .into_iter() .map(|batch| { batch.map( diff --git a/fusestore/store/src/executor/meta_handlers.rs b/fusestore/store/src/executor/meta_handlers.rs index d76ce974b1057..ce8b7a35d49d7 100644 --- a/fusestore/store/src/executor/meta_handlers.rs +++ b/fusestore/store/src/executor/meta_handlers.rs @@ -8,7 +8,8 @@ use std::convert::TryFrom; use std::sync::Arc; use common_arrow::arrow::datatypes::Schema as ArrowSchema; -use common_arrow::arrow_flight; +use common_arrow::arrow::io::ipc::write::common::IpcWriteOptions; +use common_arrow::arrow_flight::utils::flight_data_from_arrow_schema; use common_arrow::arrow_flight::FlightData; use common_exception::ErrorCode; use common_flights::meta_api_impl::CreateDatabaseAction; @@ -165,9 +166,8 @@ impl RequestHandler for ActionHandler { info!("create table: {:}: {:?}", &db_name, &table_name); - let options = common_arrow::arrow::ipc::writer::IpcWriteOptions::default(); - let flight_data: FlightData = - arrow_flight::SchemaAsIpc::new(&plan.schema.to_arrow(), &options).into(); + let options = IpcWriteOptions::default(); + let flight_data = flight_data_from_arrow_schema(&plan.schema.to_arrow(), &options); let table = Table { table_id: 0,