diff --git a/Cargo.toml b/Cargo.toml index 1e493f864c03..2db0379b0657 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,40 +17,26 @@ [workspace] exclude = ["datafusion-cli"] -members = [ - "datafusion/common", - "datafusion/core", - "datafusion/expr", - "datafusion/execution", - "datafusion/optimizer", - "datafusion/physical-expr", - "datafusion/proto", - "datafusion/proto/gen", - "datafusion/row", - "datafusion/sql", - "datafusion/substrait", - "datafusion-examples", - "test-utils", - "benchmarks", +members = ["datafusion/common", "datafusion/core", "datafusion/expr", "datafusion/execution", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/proto", "datafusion/proto/gen", "datafusion/row", "datafusion/sql", "datafusion/substrait", "datafusion-examples", "test-utils", "benchmarks", ] resolver = "2" [workspace.package] -version = "27.0.0" -edition = "2021" -readme = "README.md" authors = ["Apache Arrow "] -license = "Apache-2.0" +edition = "2021" homepage = "https://github.com/apache/arrow-datafusion" +license = "Apache-2.0" +readme = "README.md" repository = "https://github.com/apache/arrow-datafusion" rust-version = "1.64" +version = "27.0.0" [workspace.dependencies] arrow = { version = "43.0.0", features = ["prettyprint", "dyn_cmp_dict"] } -arrow-flight = { version = "43.0.0", features = ["flight-sql-experimental"] } +arrow-array = { version = "43.0.0", default-features = false, features = ["chrono-tz"] } arrow-buffer = { version = "43.0.0", default-features = false } +arrow-flight = { version = "43.0.0", features = ["flight-sql-experimental"] } arrow-schema = { version = "43.0.0", default-features = false } -arrow-array = { version = "43.0.0", default-features = false, features = ["chrono-tz"] } parquet = { version = "43.0.0", features = ["arrow", "async", "object_store"] } sqlparser = { version = "0.35", features = ["visitor"] } @@ -70,4 +56,4 @@ lto = false opt-level = 3 overflow-checks = false panic = 'unwind' -rpath = false +rpath = false \ No newline at end of file diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index d18dd38b9b44..b5f535b38d3c 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -394,9 +394,24 @@ mod tests { use std::path::Path; use super::*; - use datafusion_proto::bytes::{logical_plan_from_bytes, logical_plan_to_bytes}; + use datafusion_proto::bytes::{ + logical_plan_from_bytes, logical_plan_to_bytes, physical_plan_from_bytes, + physical_plan_to_bytes, + }; + + fn get_tpch_data_path() -> Result { + let path = + std::env::var("TPCH_DATA").unwrap_or_else(|_| "benchmarks/data".to_string()); + if !Path::new(&path).exists() { + return Err(DataFusionError::Execution(format!( + "Benchmark data not found (set TPCH_DATA env var to override): {}", + path + ))); + } + Ok(path) + } - async fn serde_round_trip(query: usize) -> Result<()> { + async fn round_trip_logical_plan(query: usize) -> Result<()> { let ctx = SessionContext::default(); let path = get_tpch_data_path()?; let opt = DataFusionBenchmarkOpt { @@ -425,125 +440,99 @@ mod tests { Ok(()) } - #[tokio::test] - async fn serde_q1() -> Result<()> { - serde_round_trip(1).await - } - - #[tokio::test] - async fn serde_q2() -> Result<()> { - serde_round_trip(2).await - } - - #[tokio::test] - async fn serde_q3() -> Result<()> { - serde_round_trip(3).await - } - - #[tokio::test] - async fn serde_q4() -> Result<()> { - serde_round_trip(4).await - } - - #[tokio::test] - async fn serde_q5() -> Result<()> { - serde_round_trip(5).await - } - - #[tokio::test] - async fn serde_q6() -> Result<()> { - serde_round_trip(6).await - } - - #[tokio::test] - async fn serde_q7() -> Result<()> { - serde_round_trip(7).await - } - - #[tokio::test] - async fn serde_q8() -> Result<()> { - serde_round_trip(8).await - } - - #[tokio::test] - async fn serde_q9() -> Result<()> { - serde_round_trip(9).await - } - - #[tokio::test] - async fn serde_q10() -> Result<()> { - serde_round_trip(10).await - } - - #[tokio::test] - async fn serde_q11() -> Result<()> { - serde_round_trip(11).await - } - - #[tokio::test] - async fn serde_q12() -> Result<()> { - serde_round_trip(12).await - } - - #[tokio::test] - async fn serde_q13() -> Result<()> { - serde_round_trip(13).await - } - - #[tokio::test] - async fn serde_q14() -> Result<()> { - serde_round_trip(14).await - } - - #[tokio::test] - async fn serde_q15() -> Result<()> { - serde_round_trip(15).await - } - - #[tokio::test] - async fn serde_q16() -> Result<()> { - serde_round_trip(16).await - } - - #[tokio::test] - async fn serde_q17() -> Result<()> { - serde_round_trip(17).await - } - - #[tokio::test] - async fn serde_q18() -> Result<()> { - serde_round_trip(18).await - } - - #[tokio::test] - async fn serde_q19() -> Result<()> { - serde_round_trip(19).await - } - - #[tokio::test] - async fn serde_q20() -> Result<()> { - serde_round_trip(20).await + async fn round_trip_physical_plan(query: usize) -> Result<()> { + let ctx = SessionContext::default(); + let path = get_tpch_data_path()?; + let opt = DataFusionBenchmarkOpt { + query: Some(query), + debug: false, + iterations: 1, + partitions: 2, + batch_size: 8192, + path: PathBuf::from(path.to_string()), + file_format: "tbl".to_string(), + mem_table: false, + output_path: None, + disable_statistics: false, + }; + register_tables(&opt, &ctx).await?; + let queries = get_query_sql(query)?; + for query in queries { + let plan = ctx.sql(&query).await?; + let plan = plan.create_physical_plan().await?; + let bytes = physical_plan_to_bytes(plan.clone())?; + let plan2 = physical_plan_from_bytes(&bytes, &ctx)?; + let plan_formatted = format!("{}", displayable(plan.as_ref()).indent(false)); + let plan2_formatted = + format!("{}", displayable(plan2.as_ref()).indent(false)); + assert_eq!(plan_formatted, plan2_formatted); + } + Ok(()) } - #[tokio::test] - async fn serde_q21() -> Result<()> { - serde_round_trip(21).await + macro_rules! test_round_trip_logical { + ($tn:ident, $query:expr) => { + #[tokio::test] + async fn $tn() -> Result<()> { + round_trip_logical_plan($query).await + } + }; } - #[tokio::test] - async fn serde_q22() -> Result<()> { - serde_round_trip(22).await + macro_rules! test_round_trip_physical { + ($tn:ident, $query:expr) => { + #[tokio::test] + async fn $tn() -> Result<()> { + round_trip_physical_plan($query).await + } + }; } - fn get_tpch_data_path() -> Result { - let path = - std::env::var("TPCH_DATA").unwrap_or_else(|_| "benchmarks/data".to_string()); - if !Path::new(&path).exists() { - return Err(DataFusionError::Execution(format!( - "Benchmark data not found (set TPCH_DATA env var to override): {}", - path - ))); - } - Ok(path) - } + // logical plan tests + test_round_trip_logical!(round_trip_logical_plan_q1, 1); + test_round_trip_logical!(round_trip_logical_plan_q2, 2); + test_round_trip_logical!(round_trip_logical_plan_q3, 3); + test_round_trip_logical!(round_trip_logical_plan_q4, 4); + test_round_trip_logical!(round_trip_logical_plan_q5, 5); + test_round_trip_logical!(round_trip_logical_plan_q6, 6); + test_round_trip_logical!(round_trip_logical_plan_q7, 7); + test_round_trip_logical!(round_trip_logical_plan_q8, 8); + test_round_trip_logical!(round_trip_logical_plan_q9, 9); + test_round_trip_logical!(round_trip_logical_plan_q10, 10); + test_round_trip_logical!(round_trip_logical_plan_q11, 11); + test_round_trip_logical!(round_trip_logical_plan_q12, 12); + test_round_trip_logical!(round_trip_logical_plan_q13, 13); + test_round_trip_logical!(round_trip_logical_plan_q14, 14); + test_round_trip_logical!(round_trip_logical_plan_q15, 15); + test_round_trip_logical!(round_trip_logical_plan_q16, 16); + test_round_trip_logical!(round_trip_logical_plan_q17, 17); + test_round_trip_logical!(round_trip_logical_plan_q18, 18); + test_round_trip_logical!(round_trip_logical_plan_q19, 19); + test_round_trip_logical!(round_trip_logical_plan_q20, 20); + test_round_trip_logical!(round_trip_logical_plan_q21, 21); + test_round_trip_logical!(round_trip_logical_plan_q22, 22); + + // physical plan tests + test_round_trip_physical!(round_trip_physical_plan_q1, 1); + test_round_trip_physical!(round_trip_physical_plan_q2, 2); + test_round_trip_physical!(round_trip_physical_plan_q3, 3); + test_round_trip_physical!(round_trip_physical_plan_q4, 4); + test_round_trip_physical!(round_trip_physical_plan_q5, 5); + test_round_trip_physical!(round_trip_physical_plan_q6, 6); + test_round_trip_physical!(round_trip_physical_plan_q7, 7); + test_round_trip_physical!(round_trip_physical_plan_q8, 8); + test_round_trip_physical!(round_trip_physical_plan_q9, 9); + test_round_trip_physical!(round_trip_physical_plan_q10, 10); + test_round_trip_physical!(round_trip_physical_plan_q11, 11); + test_round_trip_physical!(round_trip_physical_plan_q12, 12); + test_round_trip_physical!(round_trip_physical_plan_q13, 13); + test_round_trip_physical!(round_trip_physical_plan_q14, 14); + test_round_trip_physical!(round_trip_physical_plan_q15, 15); + test_round_trip_physical!(round_trip_physical_plan_q16, 16); + test_round_trip_physical!(round_trip_physical_plan_q17, 17); + test_round_trip_physical!(round_trip_physical_plan_q18, 18); + test_round_trip_physical!(round_trip_physical_plan_q19, 19); + test_round_trip_physical!(round_trip_physical_plan_q20, 20); + test_round_trip_physical!(round_trip_physical_plan_q21, 21); + test_round_trip_physical!(round_trip_physical_plan_q22, 22); } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 149c6e2c5bdf..da31a8ce65d0 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -322,9 +322,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b0122885821398cc923ece939e24d1056a2384ee719432397fa9db87230ff11" +checksum = "62b74f44609f0f91493e3082d3734d98497e094777144380ea4db9f9905dd5b6" dependencies = [ "bzip2", "flate2", @@ -346,7 +346,7 @@ checksum = "a564d521dd56509c4c47480d00b80ee55f7e385ae48db5744c67ad50c92d2ebf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -1029,17 +1029,17 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eed5fff0d93c7559121e9c72bf9c242295869396255071ff2cb1617147b608c5" dependencies = [ "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] name = "dashmap" -version = "5.4.0" +version = "5.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "907076dfda823b0b36d2a1bb5f90c96660a5bbcd7729e10727f07858f22c4edc" +checksum = "6943ae99c34386c84a470c499d3414f66502a41340aa895406e0d2e4a207b91d" dependencies = [ "cfg-if", - "hashbrown 0.12.3", + "hashbrown 0.14.0", "lock_api", "once_cell", "parking_lot_core", @@ -1197,6 +1197,7 @@ dependencies = [ "itertools 0.11.0", "lazy_static", "libc", + "log", "md-5", "paste", "petgraph", @@ -1329,9 +1330,9 @@ dependencies = [ [[package]] name = "equivalent" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88bffebc5d80432c9b140ee17875ff173a8ab62faad5b257da912bd2f6c1c0a1" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" @@ -1380,7 +1381,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ef033ed5e9bad94e55838ca0ca906db0e043f517adda0c8b79c7a8c66c93c1b5" dependencies = [ "cfg-if", - "rustix 0.38.3", + "rustix 0.38.4", "windows-sys", ] @@ -1490,7 +1491,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -1732,7 +1733,7 @@ dependencies = [ "futures-util", "http", "hyper", - "rustls 0.21.3", + "rustls 0.21.5", "tokio", "tokio-rustls 0.24.1", ] @@ -2094,9 +2095,9 @@ checksum = "61807f77802ff30975e01f4f071c8ba10c022052f98b3294119f3e615d13e5be" [[package]] name = "num" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43db66d1170d347f9a065114077f7dccb00c1b9478c89384490a3425279a4606" +checksum = "b05180d69e3da0e530ba2a1dae5110317e49e3b7f3d41be227dc5f92e49ee7af" dependencies = [ "num-bigint", "num-complex", @@ -2394,7 +2395,7 @@ checksum = "ec2e072ecce94ec471b13398d5402c188e76ac03cf74dd1a975161b23a3f6d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -2484,9 +2485,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.63" +version = "1.0.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +checksum = "78803b62cbf1f46fde80d7c0e803111524b9877184cfe7c3033659490ac7a7da" dependencies = [ "unicode-ident", ] @@ -2581,9 +2582,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.9.0" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89089e897c013b3deb627116ae56a6955a72b8bed395c9526af31c9fe528b484" +checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575" dependencies = [ "aho-corasick", "memchr", @@ -2593,9 +2594,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.3.0" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa250384981ea14565685dea16a9ccc4d1c541a13f82b9c168572264d1df8c56" +checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310" dependencies = [ "aho-corasick", "memchr", @@ -2604,9 +2605,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ab07dc67230e4a4718e70fd5c20055a4334b121f1f9db8fe63ef39ce9b8c846" +checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" [[package]] name = "reqwest" @@ -2631,7 +2632,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.3", + "rustls 0.21.5", "rustls-pemfile", "serde", "serde_json", @@ -2721,9 +2722,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.3" +version = "0.38.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac5ffa1efe7548069688cd7028f32591853cd7b5b756d41bcffd2353e4fc75b4" +checksum = "0a962918ea88d644592894bc6dc55acc6c0956488adcebbfb6e273506b7fd6e5" dependencies = [ "bitflags 2.3.3", "errno", @@ -2746,9 +2747,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.3" +version = "0.21.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b19faa85ecb5197342b54f987b142fb3e30d0c90da40f80ef4fa9a726e6676ed" +checksum = "79ea77c539259495ce8ca47f53e66ae0330a8819f67e23ac96ca02f50e7b7d36" dependencies = [ "log", "ring", @@ -2893,29 +2894,29 @@ checksum = "63134939175b3131fe4d2c131b103fd42f25ccca89423d43b5e4f267920ccf03" [[package]] name = "serde" -version = "1.0.167" +version = "1.0.171" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7daf513456463b42aa1d94cff7e0c24d682b429f020b9afa4f5ba5c40a22b237" +checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.167" +version = "1.0.171" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b69b106b68bc8054f0e974e70d19984040f8a5cf9215ca82626ea4853f82c4b9" +checksum = "389894603bd18c46fa56231694f8d827779c0951a667087194cf9de94ed24682" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] name = "serde_json" -version = "1.0.100" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f1e14e89be7aa4c4b78bdbdc9eb5bf8517829a600ae8eaa39a6e1d960b5185c" +checksum = "b5062a995d481b2308b6064e9af76011f2921c35f97b0468811ed9f6cd91dfed" dependencies = [ "itoa", "ryu", @@ -2968,9 +2969,9 @@ checksum = "62bb4feee49fdd9f707ef802e22365a35de4b7b299de4763d44bfea899442ff9" [[package]] name = "snafu" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb0656e7e3ffb70f6c39b3c2a86332bb74aa3c679da781642590f3c1118c5045" +checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" dependencies = [ "doc-comment", "snafu-derive", @@ -2978,9 +2979,9 @@ dependencies = [ [[package]] name = "snafu-derive" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "475b3bbe5245c26f2d8a6f62d67c1f30eb9fffeccee721c45d162c3ebbdf81b2" +checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" dependencies = [ "heck", "proc-macro2", @@ -3087,7 +3088,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -3109,9 +3110,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.23" +version = "2.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +checksum = "15e3fc8c0c74267e2df136e5e5fb656a464158aa57624053375eb9c8c6e25ae2" dependencies = [ "proc-macro2", "quote", @@ -3170,7 +3171,7 @@ checksum = "463fe12d7993d3b327787537ce8dd4dfa058de32fc2b195ef3cde03dc4771e8f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -3186,9 +3187,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.22" +version = "0.3.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea9e1b3cf1243ae005d9e74085d4d542f3125458f3a81af210d901dcd7411efd" +checksum = "59e399c068f43a5d116fedaf73b203fa4f9c519f17e2b34f63221d3792f81446" dependencies = [ "serde", "time-core", @@ -3203,9 +3204,9 @@ checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" [[package]] name = "time-macros" -version = "0.2.9" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "372950940a5f07bf38dbe211d7283c9e6d7327df53794992d293e534c733d09b" +checksum = "96ba15a897f3c86766b757e5ac7221554c6750054d74d5b28844fce5fb36a6c4" dependencies = [ "time-core", ] @@ -3261,7 +3262,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -3281,7 +3282,7 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.3", + "rustls 0.21.5", "tokio", ] @@ -3359,7 +3360,7 @@ checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", ] [[package]] @@ -3531,7 +3532,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", "wasm-bindgen-shared", ] @@ -3565,7 +3566,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.25", "wasm-bindgen-backend", "wasm-bindgen-shared", ] diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index d71f46a1cef9..020a24065436 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -323,10 +323,13 @@ config_namespace! { /// long runner execution, all types of joins may encounter out-of-memory errors. pub allow_symmetric_joins_without_pruning: bool, default = true - /// When set to true, file groups will be repartitioned to achieve maximum parallelism. - /// Currently supported only for Parquet format in which case - /// multiple row groups from the same file may be read concurrently. If false then each - /// row group is read serially, though different files may be read in parallel. + /// When set to `true`, file groups will be repartitioned to achieve maximum parallelism. + /// Currently Parquet and CSV formats are supported. + /// + /// If set to `true`, all files will be repartitioned evenly (i.e., a single large file + /// might be partitioned into smaller chunks) for parallel scanning. + /// If set to `false`, different files will be read in parallel, but repartitioning won't + /// happen within a single file. pub repartition_file_scans: bool, default = true /// Should DataFusion repartition data using the partitions keys to execute window diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 5bcb8bc594ff..ba986bd1cf64 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -98,7 +98,7 @@ zstd = { version = "0.12", optional = true, default-features = false } [dev-dependencies] async-trait = "0.1.53" -bigdecimal = "0.4.0" +bigdecimal = "0.3.0" criterion = { version = "0.5", features = ["async_tokio"] } csv = "1.1.6" ctor = "0.2.0" @@ -107,6 +107,7 @@ env_logger = "0.10" half = "2.2.1" postgres-protocol = "0.6.4" postgres-types = { version = "0.2.4", features = ["derive", "with-chrono-0_4"] } +regex = "1.5.4" rstest = "0.18.0" rust_decimal = { version = "1.27.0", features = ["tokio-pg"] } sqllogictest = "0.14.0" diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index c34c82a2c4d1..b284079ec6e4 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -598,8 +598,13 @@ impl DataSink for CsvSink { mod tests { use super::super::test_util::scan_format; use super::*; + use crate::arrow::util::pretty; use crate::assert_batches_eq; + use crate::datasource::file_format::file_type::FileCompressionType; + use crate::datasource::file_format::file_type::FileType; + use crate::datasource::file_format::file_type::GetExt; use crate::datasource::file_format::test_util::VariableStream; + use crate::datasource::listing::ListingOptions; use crate::physical_plan::collect; use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext}; use crate::test_util::arrow_test_data; @@ -611,6 +616,7 @@ mod tests { use futures::StreamExt; use object_store::local::LocalFileSystem; use object_store::path::Path; + use regex::Regex; use rstest::*; #[tokio::test] @@ -925,4 +931,365 @@ mod tests { ); Ok(()) } + + /// Explain the `sql` query under `ctx` to make sure the underlying csv scan is parallelized + /// e.g. "CsvExec: file_groups={2 groups:" in plan means 2 CsvExec runs concurrently + async fn count_query_csv_partitions( + ctx: &SessionContext, + sql: &str, + ) -> Result { + let df = ctx.sql(&format!("EXPLAIN {sql}")).await?; + let result = df.collect().await?; + let plan = format!("{}", &pretty::pretty_format_batches(&result)?); + + let re = Regex::new(r"CsvExec: file_groups=\{(\d+) group").unwrap(); + + if let Some(captures) = re.captures(&plan) { + if let Some(match_) = captures.get(1) { + let n_partitions = match_.as_str().parse::().unwrap(); + return Ok(n_partitions); + } + } + + Err(DataFusionError::Internal( + "query contains no CsvExec".to_string(), + )) + } + + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn test_csv_parallel_basic(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + let testdata = arrow_test_data(); + ctx.register_csv( + "aggr", + &format!("{testdata}/csv/aggregate_test_100.csv"), + CsvReadOptions::new().has_header(true), + ) + .await?; + + let query = "select sum(c2) from aggr;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "+--------------+", + "| SUM(aggr.c2) |", + "+--------------+", + "| 285 |", + "+--------------+", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(n_partitions, actual_partitions); + + Ok(()) + } + + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn test_csv_parallel_compressed(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let csv_options = CsvReadOptions::default() + .has_header(true) + .file_compression_type(FileCompressionType::GZIP) + .file_extension("csv.gz"); + let ctx = SessionContext::with_config(config); + let testdata = arrow_test_data(); + ctx.register_csv( + "aggr", + &format!("{testdata}/csv/aggregate_test_100.csv.gz"), + csv_options, + ) + .await?; + + let query = "select sum(c3) from aggr;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "+--------------+", + "| SUM(aggr.c3) |", + "+--------------+", + "| 781 |", + "+--------------+", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(1, actual_partitions); // Compressed csv won't be scanned in parallel + + Ok(()) + } + + /// Read a single empty csv file in parallel + /// + /// empty_0_byte.csv: + /// (file is empty) + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn test_csv_parallel_empty_file(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + ctx.register_csv( + "empty", + "tests/data/empty_0_byte.csv", + CsvReadOptions::new().has_header(false), + ) + .await?; + + // Require a predicate to enable repartition for the optimizer + let query = "select * from empty where random() > 0.5;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "++", + "++", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(1, actual_partitions); // Won't get partitioned if all files are empty + + Ok(()) + } + + /// Read a single empty csv file with header in parallel + /// + /// empty.csv: + /// c1,c2,c3 + #[rstest(n_partitions, case(1), case(2), case(3))] + #[tokio::test] + async fn test_csv_parallel_empty_with_header(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + ctx.register_csv( + "empty", + "tests/data/empty.csv", + CsvReadOptions::new().has_header(true), + ) + .await?; + + // Require a predicate to enable repartition for the optimizer + let query = "select * from empty where random() > 0.5;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "++", + "++", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(n_partitions, actual_partitions); + + Ok(()) + } + + /// Read multiple empty csv files in parallel + /// + /// all_empty + /// ├── empty0.csv + /// ├── empty1.csv + /// └── empty2.csv + /// + /// empty0.csv/empty1.csv/empty2.csv: + /// (file is empty) + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn test_csv_parallel_multiple_empty_files(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + let file_format = CsvFormat::default().with_has_header(false); + let listing_options = ListingOptions::new(Arc::new(file_format)) + .with_file_extension(FileType::CSV.get_ext()); + ctx.register_listing_table( + "empty", + "tests/data/empty_files/all_empty/", + listing_options, + None, + None, + ) + .await + .unwrap(); + + // Require a predicate to enable repartition for the optimizer + let query = "select * from empty where random() > 0.5;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "++", + "++", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(1, actual_partitions); // Won't get partitioned if all files are empty + + Ok(()) + } + + /// Read multiple csv files (some are empty) in parallel + /// + /// some_empty + /// ├── a_empty.csv + /// ├── b.csv + /// ├── c_empty.csv + /// ├── d.csv + /// └── e_empty.csv + /// + /// a_empty.csv/c_empty.csv/e_empty.csv: + /// (file is empty) + /// + /// b.csv/d.csv: + /// 1\n + /// 1\n + /// 1\n + /// 1\n + /// 1\n + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn test_csv_parallel_some_file_empty(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + let file_format = CsvFormat::default().with_has_header(false); + let listing_options = ListingOptions::new(Arc::new(file_format)) + .with_file_extension(FileType::CSV.get_ext()); + ctx.register_listing_table( + "empty", + "tests/data/empty_files/some_empty", + listing_options, + None, + None, + ) + .await + .unwrap(); + + // Require a predicate to enable repartition for the optimizer + let query = "select sum(column_1) from empty where column_1 > 0;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "+---------------------+", + "| SUM(empty.column_1) |", + "+---------------------+", + "| 10 |", + "+---------------------+", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(n_partitions, actual_partitions); // Won't get partitioned if all files are empty + + Ok(()) + } + + /// Parallel scan on a csv file with only 1 byte in each line + /// Testing partition byte range land on line boundaries + /// + /// one_col.csv: + /// 5\n + /// 5\n + /// (...10 rows total) + #[rstest(n_partitions, case(1), case(2), case(3), case(5), case(10), case(32))] + #[tokio::test] + async fn test_csv_parallel_one_col(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + + ctx.register_csv( + "one_col", + "tests/data/one_col.csv", + CsvReadOptions::new().has_header(false), + ) + .await?; + + let query = "select sum(column_1) from one_col where column_1 > 0;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "+-----------------------+", + "| SUM(one_col.column_1) |", + "+-----------------------+", + "| 50 |", + "+-----------------------+", + ]; + let file_size = if cfg!(target_os = "windows") { + 30 // new line on Win is '\r\n' + } else { + 20 + }; + // A 20-Byte file at most get partitioned into 20 chunks + let expected_partitions = if n_partitions <= file_size { + n_partitions + } else { + file_size + }; + assert_batches_eq!(expected, &query_result); + assert_eq!(expected_partitions, actual_partitions); + + Ok(()) + } + + /// Parallel scan on a csv file with 2 wide rows + /// The byte range of a partition might be within some line + /// + /// wode_rows.csv: + /// 1, 1, ..., 1\n (100 columns total) + /// 2, 2, ..., 2\n + #[rstest(n_partitions, case(1), case(2), case(10), case(16))] + #[tokio::test] + async fn test_csv_parallel_wide_rows(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + let ctx = SessionContext::with_config(config); + ctx.register_csv( + "wide_rows", + "tests/data/wide_rows.csv", + CsvReadOptions::new().has_header(false), + ) + .await?; + + let query = "select sum(column_1) + sum(column_33) + sum(column_50) + sum(column_77) + sum(column_100) as sum_of_5_cols from wide_rows where column_1 > 0;"; + let query_result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_query_csv_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = vec![ + "+---------------+", + "| sum_of_5_cols |", + "+---------------+", + "| 15 |", + "+---------------+", + ]; + assert_batches_eq!(expected, &query_result); + assert_eq!(n_partitions, actual_partitions); + + Ok(()) + } } diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index e27da2b41edf..fc25893d3619 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -18,6 +18,7 @@ //! Execution plan for reading CSV files use crate::datasource::file_format::file_type::FileCompressionType; +use crate::datasource::listing::FileRange; use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; @@ -39,9 +40,12 @@ use super::FileScanConfig; use bytes::{Buf, Bytes}; use futures::ready; use futures::{StreamExt, TryStreamExt}; -use object_store::{GetResult, ObjectStore}; +use object_store::local::LocalFileSystem; +use object_store::{GetOptions, GetResult, ObjectStore}; use std::any::Any; use std::fs; +use std::io::Cursor; +use std::ops::Range; use std::path::Path; use std::sync::Arc; use std::task::Poll; @@ -58,7 +62,8 @@ pub struct CsvExec { delimiter: u8, /// Execution metrics metrics: ExecutionPlanMetricsSet, - file_compression_type: FileCompressionType, + /// Compression type of the file associated with CsvExec + pub file_compression_type: FileCompressionType, } impl CsvExec { @@ -96,6 +101,34 @@ impl CsvExec { pub fn delimiter(&self) -> u8 { self.delimiter } + + /// Redistribute files across partitions according to their size + /// See comments on `repartition_file_groups()` for more detail. + /// + /// Return `None` if can't get repartitioned(empty/compressed file). + pub fn get_repartitioned( + &self, + target_partitions: usize, + repartition_file_min_size: usize, + ) -> Option { + // Parallel execution on compressed CSV file is not supported yet. + if self.file_compression_type.is_compressed() { + return None; + } + + let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( + self.base_config.file_groups.clone(), + target_partitions, + repartition_file_min_size, + ); + + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + let mut new_plan = self.clone(); + new_plan.base_config.file_groups = repartitioned_file_groups; + return Some(new_plan); + } + None + } } impl DisplayAs for CsvExec { @@ -271,14 +304,223 @@ impl CsvOpener { } } +/// Returns the position of the first newline in the byte stream, or the total length if no newline is found. +fn find_first_newline_bytes(reader: &mut R) -> Result { + let mut buffer = [0; 1]; + let mut index = 0; + + loop { + let result = reader.read(&mut buffer); + match result { + Ok(n) => { + if n == 0 { + return Ok(index); // End of file, no newline found + } + if buffer[0] == b'\n' { + return Ok(index); + } + index += 1; + } + Err(e) => { + return Err(DataFusionError::IoError(e)); + } + } + } +} + +/// Returns the offset of the first newline in the object store range [start, end), or the end offset if no newline is found. +async fn find_first_newline( + object_store: &Arc, + location: &object_store::path::Path, + start_byte: usize, + end_byte: usize, +) -> Result { + let options = GetOptions { + range: Some(Range { + start: start_byte, + end: end_byte, + }), + ..Default::default() + }; + + let offset = match object_store.get_opts(location, options).await? { + GetResult::File(_, _) => { + // Range currently is ignored for GetResult::File(...) + // Alternative get_range() will copy the whole range into memory, thus set a limit of + // max bytes to read to find the first newline + let max_line_length = 4096; // in bytes + let get_range_end_result = object_store + .get_range( + location, + Range { + start: start_byte, + end: std::cmp::min(start_byte + max_line_length, end_byte), + }, + ) + .await; + let mut decoder_tail = Cursor::new(get_range_end_result?); + find_first_newline_bytes(&mut decoder_tail)? + } + GetResult::Stream(s) => { + let mut input = s.map_err(DataFusionError::from); + let mut buffered = Bytes::new(); + + let future_index = async move { + let mut index = 0; + + loop { + if buffered.is_empty() { + match input.next().await { + Some(Ok(b)) => buffered = b, + Some(Err(e)) => return Err(e), + None => return Ok(index), + }; + } + + for byte in &buffered { + if *byte == b'\n' { + return Ok(index); + } + index += 1; + } + + buffered.advance(buffered.len()); + } + }; + future_index.await? + } + }; + Ok(offset) +} + impl FileOpener for CsvOpener { + /// Open a partitioned CSV file. + /// + /// If `file_meta.range` is `None`, the entire file is opened. + /// If `file_meta.range` is `Some(FileRange {start, end})`, this signifies that the partition + /// corresponds to the byte range [start, end) within the file. + /// + /// Note: `start` or `end` might be in the middle of some lines. In such cases, the following rules + /// are applied to determine which lines to read: + /// 1. The first line of the partition is the line in which the index of the first character >= `start`. + /// 2. The last line of the partition is the line in which the byte at position `end - 1` resides. + /// + /// Examples: + /// Consider the following partitions enclosed by braces `{}`: + /// + /// {A,1,2,3,4,5,6,7,8,9\n + /// A,1,2,3,4,5,6,7,8,9\n} + /// A,1,2,3,4,5,6,7,8,9\n + /// The lines read would be: [0, 1] + /// + /// A,{1,2,3,4,5,6,7,8,9\n + /// A,1,2,3,4,5,6,7,8,9\n + /// A},1,2,3,4,5,6,7,8,9\n + /// The lines read would be: [1, 2] fn open(&self, file_meta: FileMeta) -> Result { - let config = self.config.clone(); + // `self.config.has_header` controls whether to skip reading the 1st line header + // If the .csv file is read in parallel and this `CsvOpener` is only reading some middle + // partition, then don't skip first line + let mut csv_has_header = self.config.has_header; + if let Some(FileRange { start, .. }) = file_meta.range { + if start != 0 { + csv_has_header = false; + } + } + + let config = CsvConfig { + has_header: csv_has_header, + ..(*self.config).clone() + }; + let file_compression_type = self.file_compression_type.to_owned(); + + if file_meta.range.is_some() { + assert!( + !file_compression_type.is_compressed(), + "Reading compressed .csv in parallel is not supported" + ); + } + Ok(Box::pin(async move { - match config.object_store.get(file_meta.location()).await? { + let file_size = file_meta.object_meta.size; + // Current partition contains bytes [start_byte, end_byte) (might contain incomplete lines at boundaries) + let (start_byte, end_byte) = match file_meta.range { + None => (0, file_size), + Some(FileRange { start, end }) => { + let (start, end) = (start as usize, end as usize); + // Partition byte range is [start, end), the boundary might be in the middle of + // some line. Need to find out the exact line boundaries. + let start_delta = if start != 0 { + find_first_newline( + &config.object_store, + file_meta.location(), + start - 1, + file_size, + ) + .await? + } else { + 0 + }; + let end_delta = if end != file_size { + find_first_newline( + &config.object_store, + file_meta.location(), + end - 1, + file_size, + ) + .await? + } else { + 0 + }; + (start + start_delta, end + end_delta) + } + }; + + // For special case: If `Range` has equal `start` and `end`, object store will fetch + // the whole file + let localfs: Arc = Arc::new(LocalFileSystem::new()); + let is_localfs = localfs.type_id() == config.object_store.type_id(); + if start_byte == end_byte && !is_localfs { + return Ok(futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed()); + } + + let options = GetOptions { + range: Some(Range { + start: start_byte, + end: end_byte, + }), + ..Default::default() + }; + + match config + .object_store + .get_opts(file_meta.location(), options) + .await? + { GetResult::File(file, _) => { - let decoder = file_compression_type.convert_read(file)?; + let is_whole_file_scanned = file_meta.range.is_none(); + let decoder = if is_whole_file_scanned { + // For special case: `get_range()` will interpret `start` and `end` as the + // byte range after decompression for compressed files + file_compression_type.convert_read(file)? + } else { + // Range currently is ignored for GetResult::File(...) + let bytes = Cursor::new( + config + .object_store + .get_range( + file_meta.location(), + Range { + start: start_byte, + end: end_byte, + }, + ) + .await?, + ); + file_compression_type.convert_read(bytes)? + }; + Ok(futures::stream::iter(config.open(decoder)?).boxed()) } GetResult::Stream(s) => { diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 695e3239c561..5ca045ab2920 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -65,11 +65,13 @@ use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_physical_expr::expressions::Column; use arrow::compute::cast; +use itertools::Itertools; use log::{debug, warn}; use object_store::path::Path; use object_store::ObjectMeta; use std::{ borrow::Cow, + cmp::min, collections::HashMap, fmt::{Debug, Formatter, Result as FmtResult}, marker::PhantomData, @@ -170,7 +172,7 @@ pub struct FileScanConfig { impl FileScanConfig { /// Project the schema and the statistics on the given column indices - fn project(&self) -> (SchemaRef, Statistics, Vec) { + pub fn project(&self) -> (SchemaRef, Statistics, Vec) { if self.projection.is_none() && self.table_partition_cols.is_empty() { return ( Arc::clone(&self.file_schema), @@ -243,6 +245,81 @@ impl FileScanConfig { .collect() }) } + + /// Repartition all input files into `target_partitions` partitions, if total file size exceed + /// `repartition_file_min_size` + /// `target_partitions` and `repartition_file_min_size` directly come from configuration. + /// + /// This function only try to partition file byte range evenly, and let specific `FileOpener` to + /// do actual partition on specific data source type. (e.g. `CsvOpener` will only read lines + /// overlap with byte range but also handle boundaries to ensure all lines will be read exactly once) + pub fn repartition_file_groups( + file_groups: Vec>, + target_partitions: usize, + repartition_file_min_size: usize, + ) -> Option>> { + let flattened_files = file_groups.iter().flatten().collect::>(); + + // Perform redistribution only in case all files should be read from beginning to end + let has_ranges = flattened_files.iter().any(|f| f.range.is_some()); + if has_ranges { + return None; + } + + let total_size = flattened_files + .iter() + .map(|f| f.object_meta.size as i64) + .sum::(); + if total_size < (repartition_file_min_size as i64) || total_size == 0 { + return None; + } + + let target_partition_size = + (total_size as usize + (target_partitions) - 1) / (target_partitions); + + let current_partition_index: usize = 0; + let current_partition_size: usize = 0; + + // Partition byte range evenly for all `PartitionedFile`s + let repartitioned_files = flattened_files + .into_iter() + .scan( + (current_partition_index, current_partition_size), + |state, source_file| { + let mut produced_files = vec![]; + let mut range_start = 0; + while range_start < source_file.object_meta.size { + let range_end = min( + range_start + (target_partition_size - state.1), + source_file.object_meta.size, + ); + + let mut produced_file = source_file.clone(); + produced_file.range = Some(FileRange { + start: range_start as i64, + end: range_end as i64, + }); + produced_files.push((state.0, produced_file)); + + if state.1 + (range_end - range_start) >= target_partition_size { + state.0 += 1; + state.1 = 0; + } else { + state.1 += range_end - range_start; + } + range_start = range_end; + } + Some(produced_files) + }, + ) + .flatten() + .group_by(|(partition_idx, _)| *partition_idx) + .into_iter() + .map(|(_, group)| group.map(|(_, vals)| vals).collect_vec()) + .collect_vec(); + + Some(repartitioned_files) + } } /// The base configurations to provide when creating a physical plan for @@ -1448,4 +1525,346 @@ mod tests { extensions: None, } } + + /// Unit tests for `repartition_file_groups()` + mod repartition_file_groups_test { + use super::*; + + /// Empty file won't get partitioned + #[tokio::test] + async fn repartition_empty_file_only() { + let partitioned_file_empty = PartitionedFile::new("empty".to_string(), 0); + let file_group = vec![vec![partitioned_file_empty]]; + + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: file_group, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let partitioned_file = parquet_exec + .get_repartitioned(4, 0) + .base_config() + .file_groups + .clone(); + + assert!(partitioned_file[0][0].range.is_none()); + } + + // Repartition when there is a empty file in file groups + #[tokio::test] + async fn repartition_empty_files() { + let partitioned_file_a = PartitionedFile::new("a".to_string(), 10); + let partitioned_file_b = PartitionedFile::new("b".to_string(), 10); + let partitioned_file_empty = PartitionedFile::new("empty".to_string(), 0); + + let empty_first = vec![ + vec![partitioned_file_empty.clone()], + vec![partitioned_file_a.clone()], + vec![partitioned_file_b.clone()], + ]; + let empty_middle = vec![ + vec![partitioned_file_a.clone()], + vec![partitioned_file_empty.clone()], + vec![partitioned_file_b.clone()], + ]; + let empty_last = vec![ + vec![partitioned_file_a], + vec![partitioned_file_b], + vec![partitioned_file_empty], + ]; + + // Repartition file groups into x partitions + let expected_2 = + vec![(0, "a".to_string(), 0, 10), (1, "b".to_string(), 0, 10)]; + let expected_3 = vec![ + (0, "a".to_string(), 0, 7), + (1, "a".to_string(), 7, 10), + (1, "b".to_string(), 0, 4), + (2, "b".to_string(), 4, 10), + ]; + + //let file_groups_testset = [empty_first, empty_middle, empty_last]; + let file_groups_testset = [empty_first, empty_middle, empty_last]; + + for fg in file_groups_testset { + for (n_partition, expected) in [(2, &expected_2), (3, &expected_3)] { + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: fg.clone(), + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = file_groups_to_vec( + parquet_exec + .get_repartitioned(n_partition, 10) + .base_config() + .file_groups + .clone(), + ); + + assert_eq!(expected, &actual); + } + } + } + + #[tokio::test] + async fn repartition_single_file() { + // Single file, single partition into multiple partitions + let partitioned_file = PartitionedFile::new("a".to_string(), 123); + let single_partition = vec![vec![partitioned_file]]; + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: single_partition, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = file_groups_to_vec( + parquet_exec + .get_repartitioned(4, 10) + .base_config() + .file_groups + .clone(), + ); + let expected = vec![ + (0, "a".to_string(), 0, 31), + (1, "a".to_string(), 31, 62), + (2, "a".to_string(), 62, 93), + (3, "a".to_string(), 93, 123), + ]; + assert_eq!(expected, actual); + } + + #[tokio::test] + async fn repartition_too_much_partitions() { + // Single file, single parittion into 96 partitions + let partitioned_file = PartitionedFile::new("a".to_string(), 8); + let single_partition = vec![vec![partitioned_file]]; + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: single_partition, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = file_groups_to_vec( + parquet_exec + .get_repartitioned(96, 5) + .base_config() + .file_groups + .clone(), + ); + let expected = vec![ + (0, "a".to_string(), 0, 1), + (1, "a".to_string(), 1, 2), + (2, "a".to_string(), 2, 3), + (3, "a".to_string(), 3, 4), + (4, "a".to_string(), 4, 5), + (5, "a".to_string(), 5, 6), + (6, "a".to_string(), 6, 7), + (7, "a".to_string(), 7, 8), + ]; + assert_eq!(expected, actual); + } + + #[tokio::test] + async fn repartition_multiple_partitions() { + // Multiple files in single partition after redistribution + let partitioned_file_1 = PartitionedFile::new("a".to_string(), 40); + let partitioned_file_2 = PartitionedFile::new("b".to_string(), 60); + let source_partitions = + vec![vec![partitioned_file_1], vec![partitioned_file_2]]; + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: source_partitions, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = file_groups_to_vec( + parquet_exec + .get_repartitioned(3, 10) + .base_config() + .file_groups + .clone(), + ); + let expected = vec![ + (0, "a".to_string(), 0, 34), + (1, "a".to_string(), 34, 40), + (1, "b".to_string(), 0, 28), + (2, "b".to_string(), 28, 60), + ]; + assert_eq!(expected, actual); + } + + #[tokio::test] + async fn repartition_same_num_partitions() { + // "Rebalance" files across partitions + let partitioned_file_1 = PartitionedFile::new("a".to_string(), 40); + let partitioned_file_2 = PartitionedFile::new("b".to_string(), 60); + let source_partitions = + vec![vec![partitioned_file_1], vec![partitioned_file_2]]; + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: source_partitions, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = file_groups_to_vec( + parquet_exec + .get_repartitioned(2, 10) + .base_config() + .file_groups + .clone(), + ); + let expected = vec![ + (0, "a".to_string(), 0, 40), + (0, "b".to_string(), 0, 10), + (1, "b".to_string(), 10, 60), + ]; + assert_eq!(expected, actual); + } + + #[tokio::test] + async fn repartition_no_action_ranges() { + // No action due to Some(range) in second file + let partitioned_file_1 = PartitionedFile::new("a".to_string(), 123); + let mut partitioned_file_2 = PartitionedFile::new("b".to_string(), 144); + partitioned_file_2.range = Some(FileRange { start: 1, end: 50 }); + + let source_partitions = + vec![vec![partitioned_file_1], vec![partitioned_file_2]]; + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: source_partitions, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = parquet_exec + .get_repartitioned(65, 10) + .base_config() + .file_groups + .clone(); + assert_eq!(2, actual.len()); + } + + #[tokio::test] + async fn repartition_no_action_min_size() { + // No action due to target_partition_size + let partitioned_file = PartitionedFile::new("a".to_string(), 123); + let single_partition = vec![vec![partitioned_file]]; + let parquet_exec = ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: single_partition, + file_schema: Arc::new(Schema::empty()), + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![], + infinite_source: false, + }, + None, + None, + ); + + let actual = parquet_exec + .get_repartitioned(65, 500) + .base_config() + .file_groups + .clone(); + assert_eq!(1, actual.len()); + } + + fn file_groups_to_vec( + file_groups: Vec>, + ) -> Vec<(usize, String, i64, i64)> { + file_groups + .iter() + .enumerate() + .flat_map(|(part_idx, files)| { + files + .iter() + .map(|f| { + ( + part_idx, + f.object_meta.location.to_string(), + f.range.as_ref().unwrap().start, + f.range.as_ref().unwrap().end, + ) + }) + .collect_vec() + }) + .collect_vec() + } + } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index d8df3941f594..b61c1e1bfa0a 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -26,7 +26,6 @@ use crate::datasource::physical_plan::{ }; use crate::{ config::ConfigOptions, - datasource::listing::FileRange, error::{DataFusionError, Result}, execution::context::TaskContext, physical_optimizer::pruning::PruningPredicate, @@ -39,7 +38,6 @@ use crate::{ use datafusion_physical_expr::PhysicalSortExpr; use fmt::Debug; use std::any::Any; -use std::cmp::min; use std::fmt; use std::fs; use std::ops::Range; @@ -54,7 +52,6 @@ use datafusion_physical_expr::{ use bytes::Bytes; use futures::future::BoxFuture; use futures::{StreamExt, TryStreamExt}; -use itertools::Itertools; use log::debug; use object_store::ObjectStore; use parquet::arrow::arrow_reader::ArrowReaderOptions; @@ -72,12 +69,6 @@ mod row_groups; pub use metrics::ParquetFileMetrics; -#[derive(Default)] -struct RepartitionState { - current_partition_index: usize, - current_partition_size: usize, -} - /// Execution plan for scanning one or more Parquet partitions #[derive(Debug, Clone)] pub struct ParquetExec { @@ -253,75 +244,23 @@ impl ParquetExec { } /// Redistribute files across partitions according to their size + /// See comments on `get_file_groups_repartitioned()` for more detail. pub fn get_repartitioned( &self, target_partitions: usize, repartition_file_min_size: usize, ) -> Self { - let flattened_files = self - .base_config() - .file_groups - .iter() - .flatten() - .collect::>(); - - // Perform redistribution only in case all files should be read from beginning to end - let has_ranges = flattened_files.iter().any(|f| f.range.is_some()); - if has_ranges { - return self.clone(); - } + let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( + self.base_config.file_groups.clone(), + target_partitions, + repartition_file_min_size, + ); - let total_size = flattened_files - .iter() - .map(|f| f.object_meta.size as i64) - .sum::(); - if total_size < (repartition_file_min_size as i64) { - return self.clone(); + let mut new_plan = self.clone(); + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + new_plan.base_config.file_groups = repartitioned_file_groups; } - - let target_partition_size = - (total_size as usize + (target_partitions) - 1) / (target_partitions); - - let repartitioned_files = flattened_files - .into_iter() - .scan(RepartitionState::default(), |state, source_file| { - let mut produced_files = vec![]; - let mut range_start = 0; - while range_start < source_file.object_meta.size { - let range_end = min( - range_start - + (target_partition_size - state.current_partition_size), - source_file.object_meta.size, - ); - - let mut produced_file = source_file.clone(); - produced_file.range = Some(FileRange { - start: range_start as i64, - end: range_end as i64, - }); - produced_files.push((state.current_partition_index, produced_file)); - - if state.current_partition_size + (range_end - range_start) - >= target_partition_size - { - state.current_partition_index += 1; - state.current_partition_size = 0; - } else { - state.current_partition_size += range_end - range_start; - } - range_start = range_end; - } - Some(produced_files) - }) - .flatten() - .group_by(|(partition_idx, _)| *partition_idx) - .into_iter() - .map(|(_, group)| group.map(|(_, vals)| vals).collect_vec()) - .collect_vec(); - - let mut new_parquet_exec = self.clone(); - new_parquet_exec.base_config.file_groups = repartitioned_files; - new_parquet_exec + new_plan } } @@ -1926,242 +1865,6 @@ mod tests { assert_eq!(predicate.unwrap().to_string(), filter_phys.to_string()); } - #[tokio::test] - async fn parquet_exec_repartition_single_file() { - // Single file, single partition into multiple partitions - let partitioned_file = PartitionedFile::new("a".to_string(), 123); - let single_partition = vec![vec![partitioned_file]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: single_partition, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(4, 10) - .base_config() - .file_groups - .clone(), - ); - let expected = vec![ - (0, "a".to_string(), 0, 31), - (1, "a".to_string(), 31, 62), - (2, "a".to_string(), 62, 93), - (3, "a".to_string(), 93, 123), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn parquet_exec_repartition_too_much_partitions() { - // Single file, single parittion into 96 partitions - let partitioned_file = PartitionedFile::new("a".to_string(), 8); - let single_partition = vec![vec![partitioned_file]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: single_partition, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(96, 5) - .base_config() - .file_groups - .clone(), - ); - let expected = vec![ - (0, "a".to_string(), 0, 1), - (1, "a".to_string(), 1, 2), - (2, "a".to_string(), 2, 3), - (3, "a".to_string(), 3, 4), - (4, "a".to_string(), 4, 5), - (5, "a".to_string(), 5, 6), - (6, "a".to_string(), 6, 7), - (7, "a".to_string(), 7, 8), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn parquet_exec_repartition_multiple_partitions() { - // Multiple files in single partition after redistribution - let partitioned_file_1 = PartitionedFile::new("a".to_string(), 40); - let partitioned_file_2 = PartitionedFile::new("b".to_string(), 60); - let source_partitions = vec![vec![partitioned_file_1], vec![partitioned_file_2]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: source_partitions, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(3, 10) - .base_config() - .file_groups - .clone(), - ); - let expected = vec![ - (0, "a".to_string(), 0, 34), - (1, "a".to_string(), 34, 40), - (1, "b".to_string(), 0, 28), - (2, "b".to_string(), 28, 60), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn parquet_exec_repartition_same_num_partitions() { - // "Rebalance" files across partitions - let partitioned_file_1 = PartitionedFile::new("a".to_string(), 40); - let partitioned_file_2 = PartitionedFile::new("b".to_string(), 60); - let source_partitions = vec![vec![partitioned_file_1], vec![partitioned_file_2]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: source_partitions, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(2, 10) - .base_config() - .file_groups - .clone(), - ); - let expected = vec![ - (0, "a".to_string(), 0, 40), - (0, "b".to_string(), 0, 10), - (1, "b".to_string(), 10, 60), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn parquet_exec_repartition_no_action_ranges() { - // No action due to Some(range) in second file - let partitioned_file_1 = PartitionedFile::new("a".to_string(), 123); - let mut partitioned_file_2 = PartitionedFile::new("b".to_string(), 144); - partitioned_file_2.range = Some(FileRange { start: 1, end: 50 }); - - let source_partitions = vec![vec![partitioned_file_1], vec![partitioned_file_2]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: source_partitions, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = parquet_exec - .get_repartitioned(65, 10) - .base_config() - .file_groups - .clone(); - assert_eq!(2, actual.len()); - } - - #[tokio::test] - async fn parquet_exec_repartition_no_action_min_size() { - // No action due to target_partition_size - let partitioned_file = PartitionedFile::new("a".to_string(), 123); - let single_partition = vec![vec![partitioned_file]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: single_partition, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = parquet_exec - .get_repartitioned(65, 500) - .base_config() - .file_groups - .clone(); - assert_eq!(1, actual.len()); - } - - fn file_groups_to_vec( - file_groups: Vec>, - ) -> Vec<(usize, String, i64, i64)> { - file_groups - .iter() - .enumerate() - .flat_map(|(part_idx, files)| { - files - .iter() - .map(|f| { - ( - part_idx, - f.object_meta.location.to_string(), - f.range.as_ref().unwrap().start, - f.range.as_ref().unwrap().end, - ) - }) - .collect_vec() - }) - .collect_vec() - } - /// returns the sum of all the metrics with the specified name /// the returned set. /// diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index f94191258634..866686fe1e23 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use super::optimizer::PhysicalOptimizerRule; use crate::config::ConfigOptions; -use crate::datasource::physical_plan::ParquetExec; +use crate::datasource::physical_plan::{CsvExec, ParquetExec}; use crate::error::Result; use crate::physical_plan::Partitioning::*; use crate::physical_plan::{ @@ -257,6 +257,16 @@ fn optimize_partitions( } } + if let Some(csv_exec) = new_plan.as_any().downcast_ref::() { + if repartition_file_scans { + let repartitioned_exec_option = + csv_exec.get_repartitioned(target_partitions, repartition_file_min_size); + if let Some(repartitioned_exec) = repartitioned_exec_option { + return Ok(Transformed::Yes(Arc::new(repartitioned_exec))); + } + } + } + // Otherwise - return plan wrapped up in RepartitionExec Ok(Transformed::Yes(Arc::new(RepartitionExec::try_new( new_plan, @@ -323,6 +333,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use super::*; + use crate::datasource::file_format::file_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; @@ -345,96 +356,83 @@ mod tests { Arc::new(Schema::new(vec![Field::new("c1", DataType::Boolean, true)])) } + /// Generate FileScanConfig for file scan executors like 'ParquetExec' + fn scan_config(sorted: bool, single_file: bool) -> FileScanConfig { + let sort_exprs = vec![PhysicalSortExpr { + expr: col("c1", &schema()).unwrap(), + options: SortOptions::default(), + }]; + + let file_groups = if single_file { + vec![vec![PartitionedFile::new("x".to_string(), 100)]] + } else { + vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 200)], + ] + }; + + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema(), + file_groups, + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: if sorted { vec![sort_exprs] } else { vec![] }, + infinite_source: false, + } + } + /// Create a non sorted parquet exec fn parquet_exec() -> Arc { - Arc::new(ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema(), - file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, + Arc::new(ParquetExec::new(scan_config(false, true), None, None)) + } + + /// Create a non sorted CSV exec + fn csv_exec() -> Arc { + Arc::new(CsvExec::new( + scan_config(false, true), + false, + b',', + FileCompressionType::UNCOMPRESSED, )) } /// Create a non sorted parquet exec over two files / partitions fn parquet_exec_two_partitions() -> Arc { - Arc::new(ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema(), - file_groups: vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 200)], - ], - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, + Arc::new(ParquetExec::new(scan_config(false, false), None, None)) + } + + /// Create a non sorted csv exec over two files / partitions + fn csv_exec_two_partitions() -> Arc { + Arc::new(CsvExec::new( + scan_config(false, false), + false, + b',', + FileCompressionType::UNCOMPRESSED, )) } // Created a sorted parquet exec fn parquet_exec_sorted() -> Arc { - let sort_exprs = vec![PhysicalSortExpr { - expr: col("c1", &schema()).unwrap(), - options: SortOptions::default(), - }]; + Arc::new(ParquetExec::new(scan_config(true, true), None, None)) + } - Arc::new(ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema(), - file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![sort_exprs], - infinite_source: false, - }, - None, - None, + // Created a sorted csv exec + fn csv_exec_sorted() -> Arc { + Arc::new(CsvExec::new( + scan_config(true, true), + false, + b',', + FileCompressionType::UNCOMPRESSED, )) } // Created a sorted parquet exec with multiple files fn parquet_exec_multiple_sorted() -> Arc { - let sort_exprs = vec![PhysicalSortExpr { - expr: col("c1", &schema()).unwrap(), - options: SortOptions::default(), - }]; - - Arc::new(ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema(), - file_groups: vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ], - statistics: Statistics::default(), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![sort_exprs], - infinite_source: false, - }, - None, - None, - )) + Arc::new(ParquetExec::new(scan_config(true, false), None, None)) } fn sort_preserving_merge_exec( @@ -935,56 +933,129 @@ mod tests { #[test] fn parallelization_single_partition() -> Result<()> { - let plan = aggregate(parquet_exec()); + let plan_parquet = aggregate(parquet_exec()); + let plan_csv = aggregate(csv_exec()); - let expected = [ + let expected_parquet = [ "AggregateExec: mode=Final, gby=[], aggr=[]", "CoalescePartitionsExec", "AggregateExec: mode=Partial, gby=[], aggr=[]", "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[c1]", ]; + let expected_csv = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + "CoalescePartitionsExec", + "AggregateExec: mode=Partial, gby=[], aggr=[]", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); + Ok(()) + } + + #[test] + /// CsvExec on compressed csv file will not be partitioned + /// (Not able to decompress chunked csv file) + fn parallelization_compressed_csv() -> Result<()> { + let compression_types = [ + FileCompressionType::GZIP, + FileCompressionType::BZIP2, + FileCompressionType::XZ, + FileCompressionType::ZSTD, + FileCompressionType::UNCOMPRESSED, + ]; + + let expected_not_partitioned = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + "CoalescePartitionsExec", + "AggregateExec: mode=Partial, gby=[], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + ]; + + let expected_partitioned = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + "CoalescePartitionsExec", + "AggregateExec: mode=Partial, gby=[], aggr=[]", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[c1], has_header=false", + ]; + + for compression_type in compression_types { + let expected = if compression_type.is_compressed() { + &expected_not_partitioned[..] + } else { + &expected_partitioned[..] + }; + + let plan = aggregate(Arc::new(CsvExec::new( + scan_config(false, true), + false, + b',', + compression_type, + ))); + + assert_optimized!(expected, plan, 2, true, 10); + } Ok(()) } #[test] fn parallelization_two_partitions() -> Result<()> { - let plan = aggregate(parquet_exec_two_partitions()); + let plan_parquet = aggregate(parquet_exec_two_partitions()); + let plan_csv = aggregate(csv_exec_two_partitions()); - let expected = [ + let expected_parquet = [ "AggregateExec: mode=Final, gby=[], aggr=[]", "CoalescePartitionsExec", "AggregateExec: mode=Partial, gby=[], aggr=[]", // Plan already has two partitions "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[c1]", ]; + let expected_csv = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + "CoalescePartitionsExec", + "AggregateExec: mode=Partial, gby=[], aggr=[]", + // Plan already has two partitions + "CsvExec: file_groups={2 groups: [[x], [y]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_two_partitions_into_four() -> Result<()> { - let plan = aggregate(parquet_exec_two_partitions()); + let plan_parquet = aggregate(parquet_exec_two_partitions()); + let plan_csv = aggregate(csv_exec_two_partitions()); - let expected = [ + let expected_parquet = [ "AggregateExec: mode=Final, gby=[], aggr=[]", "CoalescePartitionsExec", "AggregateExec: mode=Partial, gby=[], aggr=[]", // Multiple source files splitted across partitions "ParquetExec: file_groups={4 groups: [[x:0..75], [x:75..100, y:0..50], [y:50..125], [y:125..200]]}, projection=[c1]", ]; + let expected_csv = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + "CoalescePartitionsExec", + "AggregateExec: mode=Partial, gby=[], aggr=[]", + // Multiple source files splitted across partitions + "CsvExec: file_groups={4 groups: [[x:0..75], [x:75..100, y:0..50], [y:50..125], [y:125..200]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 4, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 4, true, 10); + assert_optimized!(expected_csv, plan_csv, 4, true, 10); Ok(()) } #[test] fn parallelization_sorted_limit() -> Result<()> { - let plan = limit_exec(sort_exec(parquet_exec(), false)); + let plan_parquet = limit_exec(sort_exec(parquet_exec(), false)); + let plan_csv = limit_exec(sort_exec(csv_exec(), false)); - let expected = &[ + let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // data is sorted so can't repartition here @@ -992,16 +1063,26 @@ mod tests { // Doesn't parallelize for SortExec without preserve_partitioning "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1]", ]; + let expected_csv = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c1@0 ASC]", + // Doesn't parallelize for SortExec without preserve_partitioning + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_limit_with_filter() -> Result<()> { - let plan = limit_exec(filter_exec(sort_exec(parquet_exec(), false))); + let plan_parquet = limit_exec(filter_exec(sort_exec(parquet_exec(), false))); + let plan_csv = limit_exec(filter_exec(sort_exec(csv_exec(), false))); - let expected = &[ + let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", "FilterExec: c1@0", @@ -1011,16 +1092,28 @@ mod tests { // SortExec doesn't benefit from input partitioning "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1]", ]; + let expected_csv = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + "FilterExec: c1@0", + // data is sorted so can't repartition here even though + // filter would benefit from parallelism, the answers might be wrong + "SortExec: expr=[c1@0 ASC]", + // SortExec doesn't benefit from input partitioning + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_ignores_limit() -> Result<()> { - let plan = aggregate(limit_exec(filter_exec(limit_exec(parquet_exec())))); + let plan_parquet = aggregate(limit_exec(filter_exec(limit_exec(parquet_exec())))); + let plan_csv = aggregate(limit_exec(filter_exec(limit_exec(csv_exec())))); - let expected = &[ + let expected_parquet = &[ "AggregateExec: mode=Final, gby=[], aggr=[]", "CoalescePartitionsExec", "AggregateExec: mode=Partial, gby=[], aggr=[]", @@ -1036,16 +1129,34 @@ mod tests { "LocalLimitExec: fetch=100", "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1]", ]; + let expected_csv = &[ + "AggregateExec: mode=Final, gby=[], aggr=[]", + "CoalescePartitionsExec", + "AggregateExec: mode=Partial, gby=[], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c1@0", + // repartition should happen prior to the filter to maximize parallelism + "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + // Limit doesn't benefit from input partitionins - no parallelism + "LocalLimitExec: fetch=100", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_union_inputs() -> Result<()> { - let plan = union_exec(vec![parquet_exec(); 5]); + let plan_parquet = union_exec(vec![parquet_exec(); 5]); + let plan_csv = union_exec(vec![csv_exec(); 5]); - let expected = &[ + let expected_parquet = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1]", @@ -1054,40 +1165,64 @@ mod tests { "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1]", ]; + let expected_csv = &[ + "UnionExec", + // Union doesn't benefit from input partitioning - no parallelism + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, - let plan = sort_preserving_merge_exec(parquet_exec_sorted()); + let plan_parquet = sort_preserving_merge_exec(parquet_exec_sorted()); + let plan_csv = sort_preserving_merge_exec(csv_exec_sorted()); // parallelization potentially could break sort order - let expected = &[ + let expected_parquet = &[ "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC]", ]; + let expected_csv = &[ + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_sort_preserving_merge_with_union() -> Result<()> { // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) - let input = union_exec(vec![parquet_exec_sorted(); 2]); - let plan = sort_preserving_merge_exec(input); + let input_parquet = union_exec(vec![parquet_exec_sorted(); 2]); + let input_csv = union_exec(vec![csv_exec_sorted(); 2]); + let plan_parquet = sort_preserving_merge_exec(input_parquet); + let plan_csv = sort_preserving_merge_exec(input_csv); // should not repartition / sort (as the data was already sorted) - let expected = &[ + let expected_parquet = &[ "SortPreservingMergeExec: [c1@0 ASC]", "UnionExec", "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC]", ]; + let expected_csv = &[ + "SortPreservingMergeExec: [c1@0 ASC]", + "UnionExec", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } @@ -1095,31 +1230,43 @@ mod tests { fn parallelization_does_not_destroy_sort() -> Result<()> { // SortRequired // Parquet(sorted) - - let plan = sort_required_exec(parquet_exec_sorted()); + let plan_parquet = sort_required_exec(parquet_exec_sorted()); + let plan_csv = sort_required_exec(csv_exec_sorted()); // no parallelization to preserve sort order - let expected = &[ + let expected_parquet = &[ "SortRequiredExec", "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC]", ]; + let expected_csv = &[ + "SortRequiredExec", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } #[test] fn parallelization_ignores_transitively_with_projection() -> Result<()> { // sorted input - let plan = sort_preserving_merge_exec(projection_exec(parquet_exec_sorted())); + let plan_parquet = + sort_preserving_merge_exec(projection_exec(parquet_exec_sorted())); + let plan_csv = sort_preserving_merge_exec(projection_exec(csv_exec_sorted())); // data should not be repartitioned / resorted - let expected = &[ + let expected_parquet = &[ "ProjectionExec: expr=[c1@0 as c1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC]", ]; + let expected_csv = &[ + "ProjectionExec: expr=[c1@0 as c1]", + "CsvExec: file_groups={1 group: [[x]]}, projection=[c1], output_ordering=[c1@0 ASC], has_header=false", + ]; - assert_optimized!(expected, plan, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, 2, true, 10); Ok(()) } diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 60d483d8c800..5edb0476778d 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -998,7 +998,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } -/// returns physical expressions to evaluate against a batch +/// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: /// * Partial: AggregateExpr::expressions /// * Final: columns of `AggregateExpr::state_fields()` diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index beb70f1b4c55..e272b60b054a 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -16,100 +16,189 @@ // under the License. //! Hash aggregation through row format +//! +//! POC demonstration of GroupByHashApproach -use std::cmp::min; -use std::ops::Range; +use datafusion_physical_expr::{ + AggregateExpr, GroupsAccumulator, GroupsAccumulatorAdapter, +}; +use log::debug; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; use ahash::RandomState; -use arrow::row::{RowConverter, SortField}; +use arrow::row::{RowConverter, Rows, SortField}; use datafusion_physical_expr::hash_utils::create_hashes; use futures::ready; use futures::stream::{Stream, StreamExt}; -use crate::physical_plan::aggregates::utils::{ - aggr_state_schema, col_to_scalar, get_at_indices, get_optional_filters, - read_as_batch, slice_and_maybe_filter, ExecutionState, GroupState, -}; use crate::physical_plan::aggregates::{ evaluate_group_by, evaluate_many, evaluate_optional, group_schema, AggregateMode, - PhysicalGroupBy, RowAccumulatorItem, + PhysicalGroupBy, }; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; -use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; +use crate::physical_plan::{aggregates, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; -use arrow::compute::cast; -use arrow::datatypes::DataType; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; -use datafusion_common::cast::as_boolean_array; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_expr::Accumulator; -use datafusion_row::accessor::RowAccessor; -use datafusion_row::layout::RowLayout; use hashbrown::raw::RawTable; -use itertools::izip; + +#[derive(Debug, Clone)] +/// This object tracks the aggregation phase (input/output) +pub(crate) enum ExecutionState { + ReadingInput, + /// When producing output, the remaining rows to output are stored + /// here and are sliced off as needed in batch_size chunks + ProducingOutput(RecordBatch), + Done, +} use super::AggregateExec; -/// Grouping aggregate with row-format aggregation states inside. +/// Hash based Grouping Aggregator +/// +/// # Design Goals +/// +/// This structure is designed so that updating the aggregates can be +/// vectorized (done in a tight loop) without allocations. The +/// accumulator state is *not* managed by this operator (e.g in the +/// hash table) and instead is delegated to the individual +/// accumulators which have type specialized inner loops that perform +/// the aggregation. +/// +/// # Architecture +/// +/// ```text +/// +/// stores "group stores group values, internally stores aggregate +/// indexes" in arrow_row format values, for all groups /// -/// For each aggregation entry, we use: -/// - [Arrow-row] represents grouping keys for fast hash computation and comparison directly on raw bytes. -/// - [WordAligned] row to store aggregation state, designed to be CPU-friendly when updates over every field are often. +/// ┌─────────────┐ ┌────────────┐ ┌──────────────┐ ┌──────────────┐ +/// │ ┌─────┐ │ │ ┌────────┐ │ │┌────────────┐│ │┌────────────┐│ +/// │ │ 5 │ │ ┌────┼▶│ "A" │ │ ││accumulator ││ ││accumulator ││ +/// │ ├─────┤ │ │ │ ├────────┤ │ ││ 0 ││ ││ N ││ +/// │ │ 9 │ │ │ │ │ "Z" │ │ ││ ┌────────┐ ││ ││ ┌────────┐ ││ +/// │ └─────┘ │ │ │ └────────┘ │ ││ │ state │ ││ ││ │ state │ ││ +/// │ ... │ │ │ │ ││ │┌─────┐ │ ││ ... ││ │┌─────┐ │ ││ +/// │ ┌─────┐ │ │ │ ... │ ││ │├─────┤ │ ││ ││ │├─────┤ │ ││ +/// │ │ 1 │───┼─┘ │ │ ││ │└─────┘ │ ││ ││ │└─────┘ │ ││ +/// │ ├─────┤ │ │ │ ││ │ │ ││ ││ │ │ ││ +/// │ │ 13 │───┼─┐ │ ┌────────┐ │ ││ │ ... │ ││ ││ │ ... │ ││ +/// │ └─────┘ │ └────┼▶│ "Q" │ │ ││ │ │ ││ ││ │ │ ││ +/// └─────────────┘ │ └────────┘ │ ││ │┌─────┐ │ ││ ││ │┌─────┐ │ ││ +/// │ │ ││ │└─────┘ │ ││ ││ │└─────┘ │ ││ +/// └────────────┘ ││ └────────┘ ││ ││ └────────┘ ││ +/// │└────────────┘│ │└────────────┘│ +/// └──────────────┘ └──────────────┘ /// -/// The architecture is the following: +/// map group_values accumulators +/// (Hash Table) /// -/// 1. For each input RecordBatch, update aggregation states corresponding to all appeared grouping keys. -/// 2. At the end of the aggregation (e.g. end of batches in a partition), the accumulator converts its state to a RecordBatch of a single row -/// 3. The RecordBatches of all accumulators are merged (`concatenate` in `rust/arrow`) together to a single RecordBatch. -/// 4. The state's RecordBatch is `merge`d to a new state -/// 5. The state is mapped to the final value +/// ``` /// -/// [WordAligned]: datafusion_row::layout +/// For example, given a query like `COUNT(x), SUM(y) ... GROUP BY z`, +/// [`group_values`] will store the distinct values of `z`. There will +/// be one accumulator for `COUNT(x)`, specialized for the data type +/// of `x` and one accumulator for `SUM(y)`, specialized for the data +/// type of `y`. +/// +/// # Description +/// +/// The hash table does not store any aggregate state inline. It only +/// stores "group indices", one for each (distinct) group value. The +/// accumulators manage the in-progress aggregate state for each +/// group, and the group values themselves are stored in +/// [`group_values`] at the corresponding group index. +/// +/// The accumulator state (e.g partial sums) is managed by and stored +/// by a [`GroupsAccumulator`] accumulator. There is one accumulator +/// per aggregate expression (COUNT, AVG, etc) in the +/// stream. Internally, each `GroupsAccumulator` manages the state for +/// multiple groups, and is passed `group_indexes` during update. Note +/// The accumulator state is not managed by this operator (e.g in the +/// hash table). +/// +/// [`group_values`]: Self::group_values pub(crate) struct GroupedHashAggregateStream { schema: SchemaRef, input: SendableRecordBatchStream, mode: AggregateMode, - normal_aggr_expr: Vec>, - /// Aggregate expressions not supporting row accumulation - normal_aggregate_expressions: Vec>>, - /// Filter expression for each normal aggregate expression - normal_filter_expressions: Vec>>, - - /// Aggregate expressions supporting row accumulation - row_aggregate_expressions: Vec>>, - /// Filter expression for each row aggregate expression - row_filter_expressions: Vec>>, - row_accumulators: Vec, + /// Accumulators, one for each `AggregateExpr` in the query + /// + /// For example, if the query has aggregates, `SUM(x)`, + /// `COUNT(y)`, there will be two accumulators, each one + /// specialized for that particular aggregate and its input types + accumulators: Vec>, + + /// Arguments to pass to each accumulator. + /// + /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` + /// + /// The argument to each accumulator is itself a `Vec` because + /// some aggregates such as `CORR` can accept more than one + /// argument. + aggregate_arguments: Vec>>, + + /// Optional filter expression to evaluate, one for each for + /// accumulator. If present, only those rows for which the filter + /// evaluate to true should be included in the aggregate results. + /// + /// For example, for an aggregate like `SUM(x FILTER x > 100)`, + /// the filter expression is `x > 100`. + filter_expressions: Vec>>, + + /// Converter for the group values row_converter: RowConverter, - row_aggr_schema: SchemaRef, - row_aggr_layout: Arc, + /// GROUP BY expressions group_by: PhysicalGroupBy, - aggr_state: AggregationState, + /// The memory reservation for this grouping + reservation: MemoryReservation, + + /// Logically maps group values to a group_index in + /// [`Self::group_values`] and in each accumulator + /// + /// Uses the raw API of hashbrown to avoid actually storing the + /// keys (group values) in the table + /// + /// keys: u64 hashes of the GroupValue + /// values: (hash, group_index) + map: RawTable<(u64, usize)>, + + /// The actual group by values, stored in arrow [`Row`] format. + /// `group_values[i]` holds the group value for group_index `i`. + /// + /// The row format is used to compare group keys quickly and store + /// them efficiently in memory. Quick comparison is especially + /// important for multi-column group keys. + /// + /// [`Row`]: arrow::row::Row + group_values: Rows, + + /// scratch space for the current input [`RecordBatch`] being + /// processed. The reason this is a field is so it can be reused + /// for all input batches, avoiding the need to reallocate Vecs on + /// each input. + scratch_space: ScratchSpace, + + /// Tracks if this stream is generating input or output exec_state: ExecutionState, + + /// Execution metrics baseline_metrics: BaselineMetrics, + + /// Random state for creating hashes random_state: RandomState, - /// size to be used for resulting RecordBatches + + /// max rows in output RecordBatches batch_size: usize, - /// threshold for using `ScalarValue`s to update - /// accumulators during high-cardinality aggregations for each input batch. - scalar_update_factor: usize, - /// if the result is chunked into batches, - /// last offset is preserved for continuation. - row_group_skip_position: usize, - /// keeps range for each accumulator in the field - /// first element in the array corresponds to normal accumulators - /// second element in the array corresponds to row accumulators - indices: [Vec>; 2], } impl GroupedHashAggregateStream { @@ -119,71 +208,39 @@ impl GroupedHashAggregateStream { context: Arc, partition: usize, ) -> Result { + debug!("Creating GroupedHashAggregateStream"); let agg_schema = Arc::clone(&agg.schema); let agg_group_by = agg.group_by.clone(); let agg_filter_expr = agg.filter_expr.clone(); let batch_size = context.session_config().batch_size(); - let scalar_update_factor = context.session_config().agg_scalar_update_factor(); let input = agg.input.execute(partition, Arc::clone(&context))?; let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let timer = baseline_metrics.elapsed_compute().timer(); - let mut start_idx = agg_group_by.expr.len(); - let mut row_aggr_expr = vec![]; - let mut row_agg_indices = vec![]; - let mut row_aggregate_expressions = vec![]; - let mut row_filter_expressions = vec![]; - let mut normal_aggr_expr = vec![]; - let mut normal_agg_indices = vec![]; - let mut normal_aggregate_expressions = vec![]; - let mut normal_filter_expressions = vec![]; - // The expressions to evaluate the batch, one vec of expressions per aggregation. - // Assuming create_schema() always puts group columns in front of aggregation columns, we set - // col_idx_base to the group expression count. - let all_aggregate_expressions = - aggregates::aggregate_expressions(&agg.aggr_expr, &agg.mode, start_idx)?; + let aggregate_exprs = agg.aggr_expr.clone(); + + // arguments for each aggregate, one vec of expressions per + // aggregate + let aggregate_arguments = aggregates::aggregate_expressions( + &agg.aggr_expr, + &agg.mode, + agg_group_by.expr.len(), + )?; + let filter_expressions = match agg.mode { AggregateMode::Partial | AggregateMode::Single => agg_filter_expr, AggregateMode::Final | AggregateMode::FinalPartitioned => { vec![None; agg.aggr_expr.len()] } }; - for ((expr, others), filter) in agg - .aggr_expr - .iter() - .zip(all_aggregate_expressions.into_iter()) - .zip(filter_expressions.into_iter()) - { - let n_fields = match agg.mode { - // In partial aggregation, we keep additional fields in order to successfully - // merge aggregation results downstream. - AggregateMode::Partial => expr.state_fields()?.len(), - _ => 1, - }; - // Stores range of each expression: - let aggr_range = Range { - start: start_idx, - end: start_idx + n_fields, - }; - if expr.row_accumulator_supported() { - row_aggregate_expressions.push(others); - row_filter_expressions.push(filter.clone()); - row_agg_indices.push(aggr_range); - row_aggr_expr.push(expr.clone()); - } else { - normal_aggregate_expressions.push(others); - normal_filter_expressions.push(filter.clone()); - normal_agg_indices.push(aggr_range); - normal_aggr_expr.push(expr.clone()); - } - start_idx += n_fields; - } - let row_accumulators = aggregates::create_row_accumulators(&row_aggr_expr)?; - - let row_aggr_schema = aggr_state_schema(&row_aggr_expr); + // Instantiate the accumulators + let accumulators: Vec<_> = aggregate_exprs + .iter() + .map(create_group_accumulator) + .collect::>()?; let group_schema = group_schema(&agg_schema, agg_group_by.expr.len()); let row_converter = RowConverter::new( @@ -194,14 +251,10 @@ impl GroupedHashAggregateStream { .collect(), )?; - let row_aggr_layout = Arc::new(RowLayout::new(&row_aggr_schema)); - let name = format!("GroupedHashAggregateStream[{partition}]"); - let aggr_state = AggregationState { - reservation: MemoryConsumer::new(name).register(context.memory_pool()), - map: RawTable::with_capacity(0), - group_states: Vec::with_capacity(0), - }; + let reservation = MemoryConsumer::new(name).register(context.memory_pool()); + let map = RawTable::with_capacity(0); + let group_values = row_converter.empty_rows(0, 0); timer.done(); @@ -211,28 +264,43 @@ impl GroupedHashAggregateStream { schema: agg_schema, input, mode: agg.mode, - normal_aggr_expr, - normal_aggregate_expressions, - normal_filter_expressions, - row_aggregate_expressions, - row_filter_expressions, - row_accumulators, + accumulators, + aggregate_arguments, + filter_expressions, row_converter, - row_aggr_schema, - row_aggr_layout, group_by: agg_group_by, - aggr_state, + reservation, + map, + group_values, + scratch_space: ScratchSpace::new(), exec_state, baseline_metrics, random_state: Default::default(), batch_size, - scalar_update_factor, - row_group_skip_position: 0, - indices: [normal_agg_indices, row_agg_indices], }) } } +/// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if +/// that is supported by the aggregate, or a +/// [`GroupsAccumulatorAdapter`] if not. +fn create_group_accumulator( + agg_expr: &Arc, +) -> Result> { + if agg_expr.groups_accumulator_supported() { + agg_expr.create_groups_accumulator() + } else { + // Note in the log when the slow path is used + debug!( + "Creating GroupsAccumulatorAdapter for {}: {agg_expr:?}", + agg_expr.name() + ); + let agg_expr_captured = agg_expr.clone(); + let factory = move || agg_expr_captured.create_accumulator(); + Ok(Box::new(GroupsAccumulatorAdapter::new(factory))) + } +} + impl Stream for GroupedHashAggregateStream { type Item = Result; @@ -243,7 +311,8 @@ impl Stream for GroupedHashAggregateStream { let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); loop { - match self.exec_state { + let exec_state = self.exec_state.clone(); + match exec_state { ExecutionState::ReadingInput => { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate @@ -252,11 +321,15 @@ impl Stream for GroupedHashAggregateStream { let result = self.group_aggregate_batch(batch); timer.done(); - // allocate memory - // This happens AFTER we actually used the memory, but simplifies the whole accounting and we are OK with - // overshooting a bit. Also this means we either store the whole record batch or not. + // allocate memory AFTER we actually used + // the memory, which simplifies the whole + // accounting and we are OK with + // overshooting a bit. + // + // Also this means we either store the + // whole record batch or not. let result = result.and_then(|allocated| { - self.aggr_state.reservation.try_grow(allocated) + self.reservation.try_grow(allocated) }); if let Err(e) = result { @@ -267,32 +340,36 @@ impl Stream for GroupedHashAggregateStream { Some(Err(e)) => return Poll::Ready(Some(Err(e))), // inner is done, producing output None => { - self.exec_state = ExecutionState::ProducingOutput; + let timer = elapsed_compute.timer(); + match self.create_batch_from_map() { + Ok(batch) => { + self.exec_state = + ExecutionState::ProducingOutput(batch) + } + Err(e) => return Poll::Ready(Some(Err(e))), + } + timer.done(); } } } - ExecutionState::ProducingOutput => { - let timer = elapsed_compute.timer(); - let result = self.create_batch_from_map(); - - timer.done(); - self.row_group_skip_position += self.batch_size; - - match result { - // made output - Ok(Some(result)) => { - let batch = result.record_output(&self.baseline_metrics); - return Poll::Ready(Some(Ok(batch))); - } - // end of output - Ok(None) => { - self.exec_state = ExecutionState::Done; - } - // error making output - Err(error) => return Poll::Ready(Some(Err(error))), - } + ExecutionState::ProducingOutput(batch) => { + // slice off a part of the batch, if needed + let output_batch = if batch.num_rows() <= self.batch_size { + self.exec_state = ExecutionState::Done; + batch + } else { + // output first batch_size rows + let num_remaining = batch.num_rows() - self.batch_size; + let remaining = batch.slice(self.batch_size, num_remaining); + self.exec_state = ExecutionState::ProducingOutput(remaining); + batch.slice(0, self.batch_size) + }; + return Poll::Ready(Some(Ok( + output_batch.record_output(&self.baseline_metrics) + ))); } + ExecutionState::Done => return Poll::Ready(None), } } @@ -306,460 +383,194 @@ impl RecordBatchStream for GroupedHashAggregateStream { } impl GroupedHashAggregateStream { - // Update the row_aggr_state according to groub_by values (result of group_by_expressions) + /// Calculates the group indices for each input row of + /// `group_values`. + /// + /// At the return of this function, + /// `self.scratch_space.current_group_indices` has the same number + /// of entries as each array in `group_values` and holds the + /// correct group_index for that row. + /// + /// This is one of the core hot loops in the algorithm fn update_group_state( &mut self, group_values: &[ArrayRef], allocated: &mut usize, - ) -> Result> { + ) -> Result<()> { + // Convert the group keys into the row format + // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available let group_rows = self.row_converter.convert_columns(group_values)?; let n_rows = group_rows.num_rows(); - // 1.1 construct the key from the group values - // 1.2 construct the mapping key if it does not exist - // 1.3 add the row' index to `indices` - // track which entries in `aggr_state` have rows in this batch to aggregate - let mut groups_with_rows = vec![]; + // track memory used + let group_values_size_pre = self.group_values.size(); + let scratch_size_pre = self.scratch_space.size(); - // 1.1 Calculate the group keys for the group values - let mut batch_hashes = vec![0; n_rows]; - create_hashes(group_values, &self.random_state, &mut batch_hashes)?; + // tracks to which group each of the input rows belongs + let group_indices = &mut self.scratch_space.current_group_indices; + group_indices.clear(); - let AggregationState { - map, group_states, .. - } = &mut self.aggr_state; + // 1.1 Calculate the group keys for the group values + let batch_hashes = &mut self.scratch_space.hashes_buffer; + batch_hashes.clear(); + batch_hashes.resize(n_rows, 0); + create_hashes(group_values, &self.random_state, batch_hashes)?; - for (row, hash) in batch_hashes.into_iter().enumerate() { - let entry = map.get_mut(hash, |(_hash, group_idx)| { + for (row, &hash) in batch_hashes.iter().enumerate() { + let entry = self.map.get_mut(hash, |(_hash, group_idx)| { // verify that a group that we are inserting with hash is // actually the same key value as the group in // existing_idx (aka group_values @ row) - let group_state = &group_states[*group_idx]; - - group_rows.row(row) == group_state.group_by_values.row() + group_rows.row(row) == self.group_values.row(*group_idx) }); - match entry { - // Existing entry for this group value - Some((_hash, group_idx)) => { - let group_state = &mut group_states[*group_idx]; - - // 1.3 - if group_state.indices.is_empty() { - groups_with_rows.push(*group_idx); - }; - - group_state.indices.push_accounted(row as u32, allocated); // remember this row - } - // 1.2 Need to create new entry + let group_idx = match entry { + // Existing group_index for this group value + Some((_hash, group_idx)) => *group_idx, + // 1.2 Need to create new entry for the group None => { - let accumulator_set = - aggregates::create_accumulators(&self.normal_aggr_expr)?; - // Add new entry to group_states and save newly created index - let group_state = GroupState { - group_by_values: group_rows.row(row).owned(), - aggregation_buffer: vec![ - 0; - self.row_aggr_layout.fixed_part_width() - ], - accumulator_set, - indices: vec![row as u32], // 1.3 - }; - let group_idx = group_states.len(); - - // NOTE: do NOT include the `GroupState` struct size in here because this is captured by - // `group_states` (see allocation down below) - *allocated += std::mem::size_of_val(&group_state.group_by_values) - + (std::mem::size_of::() - * group_state.aggregation_buffer.capacity()) - + (std::mem::size_of::() * group_state.indices.capacity()); - - // Allocation done by normal accumulators - *allocated += (std::mem::size_of::>() - * group_state.accumulator_set.capacity()) - + group_state - .accumulator_set - .iter() - .map(|accu| accu.size()) - .sum::(); + // Add new entry to aggr_state and save newly created index + let group_idx = self.group_values.num_rows(); + self.group_values.push(group_rows.row(row)); // for hasher function, use precomputed hash value - map.insert_accounted( + self.map.insert_accounted( (hash, group_idx), |(hash, _group_index)| *hash, allocated, ); - - group_states.push_accounted(group_state, allocated); - - groups_with_rows.push(group_idx); + group_idx } }; + group_indices.push(group_idx); } - Ok(groups_with_rows) - } - // Update the accumulator results, according to row_aggr_state. - #[allow(clippy::too_many_arguments)] - fn update_accumulators_using_batch( - &mut self, - groups_with_rows: &[usize], - offsets: &[usize], - row_values: &[Vec], - normal_values: &[Vec], - row_filter_values: &[Option], - normal_filter_values: &[Option], - allocated: &mut usize, - ) -> Result<()> { - // 2.1 for each key in this batch - // 2.2 for each aggregation - // 2.3 `slice` from each of its arrays the keys' values - // 2.4 update / merge the accumulator with the values - // 2.5 clear indices - groups_with_rows - .iter() - .zip(offsets.windows(2)) - .try_for_each(|(group_idx, offsets)| { - let group_state = &mut self.aggr_state.group_states[*group_idx]; - // 2.2 - // Process row accumulators - self.row_accumulators - .iter_mut() - .zip(row_values.iter()) - .zip(row_filter_values.iter()) - .try_for_each(|((accumulator, aggr_array), filter_opt)| { - let values = slice_and_maybe_filter( - aggr_array, - filter_opt.as_ref(), - offsets, - )?; - let mut state_accessor = - RowAccessor::new_from_layout(self.row_aggr_layout.clone()); - state_accessor - .point_to(0, group_state.aggregation_buffer.as_mut_slice()); - match self.mode { - AggregateMode::Partial | AggregateMode::Single => { - accumulator.update_batch(&values, &mut state_accessor) - } - AggregateMode::FinalPartitioned | AggregateMode::Final => { - // note: the aggregation here is over states, not values, thus the merge - accumulator.merge_batch(&values, &mut state_accessor) - } - } - })?; - // normal accumulators - group_state - .accumulator_set - .iter_mut() - .zip(normal_values.iter()) - .zip(normal_filter_values.iter()) - .try_for_each(|((accumulator, aggr_array), filter_opt)| { - let values = slice_and_maybe_filter( - aggr_array, - filter_opt.as_ref(), - offsets, - )?; - let size_pre = accumulator.size(); - let res = match self.mode { - AggregateMode::Partial | AggregateMode::Single => { - accumulator.update_batch(&values) - } - AggregateMode::FinalPartitioned | AggregateMode::Final => { - // note: the aggregation here is over states, not values, thus the merge - accumulator.merge_batch(&values) - } - }; - let size_post = accumulator.size(); - *allocated += size_post.saturating_sub(size_pre); - res - }) - // 2.5 - .and({ - group_state.indices.clear(); - Ok(()) - }) - })?; - Ok(()) - } + // account for memory growth in scratch space + *allocated += self.scratch_space.size(); + *allocated -= scratch_size_pre; // subtract after adding to avoid underflow - // Update the accumulator results, according to row_aggr_state. - fn update_accumulators_using_scalar( - &mut self, - groups_with_rows: &[usize], - row_values: &[Vec], - row_filter_values: &[Option], - ) -> Result<()> { - let filter_bool_array = row_filter_values - .iter() - .map(|filter_opt| match filter_opt { - Some(f) => Ok(Some(as_boolean_array(f)?)), - None => Ok(None), - }) - .collect::>>()?; - - for group_idx in groups_with_rows { - let group_state = &mut self.aggr_state.group_states[*group_idx]; - let mut state_accessor = - RowAccessor::new_from_layout(self.row_aggr_layout.clone()); - state_accessor.point_to(0, group_state.aggregation_buffer.as_mut_slice()); - for idx in &group_state.indices { - for (accumulator, values_array, filter_array) in izip!( - self.row_accumulators.iter_mut(), - row_values.iter(), - filter_bool_array.iter() - ) { - if values_array.len() == 1 { - let scalar_value = - col_to_scalar(&values_array[0], filter_array, *idx as usize)?; - accumulator.update_scalar(&scalar_value, &mut state_accessor)?; - } else { - let scalar_values = values_array - .iter() - .map(|array| { - col_to_scalar(array, filter_array, *idx as usize) - }) - .collect::>>()?; - accumulator - .update_scalar_values(&scalar_values, &mut state_accessor)?; - } - } - } - // clear the group indices in this group - group_state.indices.clear(); - } + // account for any memory increase used to store group_values + *allocated += self.group_values.size(); + *allocated -= group_values_size_pre; // subtract after adding to avoid underflow Ok(()) } /// Perform group-by aggregation for the given [`RecordBatch`]. /// - /// If successful, this returns the additional number of bytes that were allocated during this process. - /// + /// If successful, returns the additional amount of memory, in + /// bytes, that were allocated during this process. fn group_aggregate_batch(&mut self, batch: RecordBatch) -> Result { - // Evaluate the grouping expressions: + // Evaluate the grouping expressions let group_by_values = evaluate_group_by(&self.group_by, &batch)?; + // Keep track of memory allocated: let mut allocated = 0usize; // Evaluate the aggregation expressions. - // We could evaluate them after the `take`, but since we need to evaluate all - // of them anyways, it is more performant to do it while they are together. - let row_aggr_input_values = - evaluate_many(&self.row_aggregate_expressions, &batch)?; - let normal_aggr_input_values = - evaluate_many(&self.normal_aggregate_expressions, &batch)?; - let row_filter_values = evaluate_optional(&self.row_filter_expressions, &batch)?; - let normal_filter_values = - evaluate_optional(&self.normal_filter_expressions, &batch)?; + let input_values = evaluate_many(&self.aggregate_arguments, &batch)?; + + // Evaluate the filter expressions, if any, against the inputs + let filter_values = evaluate_optional(&self.filter_expressions, &batch)?; let row_converter_size_pre = self.row_converter.size(); + for group_values in &group_by_values { - let groups_with_rows = - self.update_group_state(group_values, &mut allocated)?; - // Decide the accumulators update mode, use scalar value to update the accumulators when all of the conditions are meet: - // 1) The aggregation mode is Partial or Single - // 2) There is not normal aggregation expressions - // 3) The number of affected groups is high (entries in `aggr_state` have rows need to update). Usually the high cardinality case - if matches!(self.mode, AggregateMode::Partial | AggregateMode::Single) - && normal_aggr_input_values.is_empty() - && normal_filter_values.is_empty() - && groups_with_rows.len() >= batch.num_rows() / self.scalar_update_factor - { - self.update_accumulators_using_scalar( - &groups_with_rows, - &row_aggr_input_values, - &row_filter_values, - )?; - } else { - // Collect all indices + offsets based on keys in this vec - let mut batch_indices: UInt32Builder = UInt32Builder::with_capacity(0); - let mut offsets = vec![0]; - let mut offset_so_far = 0; - for &group_idx in groups_with_rows.iter() { - let indices = &self.aggr_state.group_states[group_idx].indices; - batch_indices.append_slice(indices); - offset_so_far += indices.len(); - offsets.push(offset_so_far); + // calculate the group indices for each input row + self.update_group_state(group_values, &mut allocated)?; + let group_indices = &self.scratch_space.current_group_indices; + + // Gather the inputs to call the actual accumulator + let t = self + .accumulators + .iter_mut() + .zip(input_values.iter()) + .zip(filter_values.iter()); + + let total_num_groups = self.group_values.num_rows(); + + for ((acc, values), opt_filter) in t { + let acc_size_pre = acc.size(); + let opt_filter = opt_filter.as_ref().map(|filter| filter.as_boolean()); + + // Call the appropriate method on each aggregator with + // the entire input row and the relevant group indexes + match self.mode { + AggregateMode::Partial | AggregateMode::Single => { + acc.update_batch( + values, + group_indices, + opt_filter, + total_num_groups, + )?; + } + AggregateMode::FinalPartitioned | AggregateMode::Final => { + // if aggregation is over intermediate states, + // use merge + acc.merge_batch( + values, + group_indices, + opt_filter, + total_num_groups, + )?; + } } - let batch_indices = batch_indices.finish(); - - let row_values = get_at_indices(&row_aggr_input_values, &batch_indices)?; - let normal_values = - get_at_indices(&normal_aggr_input_values, &batch_indices)?; - let row_filter_values = - get_optional_filters(&row_filter_values, &batch_indices); - let normal_filter_values = - get_optional_filters(&normal_filter_values, &batch_indices); - self.update_accumulators_using_batch( - &groups_with_rows, - &offsets, - &row_values, - &normal_values, - &row_filter_values, - &normal_filter_values, - &mut allocated, - )?; + + allocated += acc.size(); + allocated -= acc_size_pre; } } - allocated += self - .row_converter - .size() - .saturating_sub(row_converter_size_pre); - Ok(allocated) - } -} - -/// The state of all the groups -pub(crate) struct AggregationState { - pub reservation: MemoryReservation, - - /// Logically maps group values to an index in `group_states` - /// - /// Uses the raw API of hashbrown to avoid actually storing the - /// keys in the table - /// - /// keys: u64 hashes of the GroupValue - /// values: (hash, index into `group_states`) - pub map: RawTable<(u64, usize)>, - - /// State for each group - pub group_states: Vec, -} + allocated += self.row_converter.size(); + allocated -= row_converter_size_pre; -impl std::fmt::Debug for AggregationState { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - // hashes are not store inline, so could only get values - let map_string = "RawTable"; - f.debug_struct("AggregationState") - .field("map", &map_string) - .field("group_states", &self.group_states) - .finish() + Ok(allocated) } -} -impl GroupedHashAggregateStream { - /// Create a RecordBatch with all group keys and accumulator' states or values. - fn create_batch_from_map(&mut self) -> Result> { - let skip_items = self.row_group_skip_position; - if skip_items > self.aggr_state.group_states.len() { - return Ok(None); - } - if self.aggr_state.group_states.is_empty() { + /// Create an output RecordBatch with all group keys and accumulator states/values + fn create_batch_from_map(&mut self) -> Result { + if self.group_values.num_rows() == 0 { let schema = self.schema.clone(); - return Ok(Some(RecordBatch::new_empty(schema))); + return Ok(RecordBatch::new_empty(schema)); } - let end_idx = min( - skip_items + self.batch_size, - self.aggr_state.group_states.len(), - ); - let group_state_chunk = &self.aggr_state.group_states[skip_items..end_idx]; - - if group_state_chunk.is_empty() { - let schema = self.schema.clone(); - return Ok(Some(RecordBatch::new_empty(schema))); + // First output rows are the groups + let groups_rows = self.group_values.iter(); + let mut output: Vec = self.row_converter.convert_rows(groups_rows)?; + + // Next output each aggregate value, from the accumulators + for acc in self.accumulators.iter_mut() { + match self.mode { + AggregateMode::Partial => output.extend(acc.state()?), + AggregateMode::Final + | AggregateMode::FinalPartitioned + | AggregateMode::Single => output.push(acc.evaluate()?), + } } - // Buffers for each distinct group (i.e. row accumulator memories) - let mut state_buffers = group_state_chunk - .iter() - .map(|gs| gs.aggregation_buffer.clone()) - .collect::>(); - - let output_fields = self.schema.fields(); - // Store row accumulator results (either final output or intermediate state): - let row_columns = match self.mode { - AggregateMode::Partial => { - read_as_batch(&state_buffers, &self.row_aggr_schema) - } - AggregateMode::Final - | AggregateMode::FinalPartitioned - | AggregateMode::Single => { - let mut results = vec![]; - for (idx, acc) in self.row_accumulators.iter().enumerate() { - let mut state_accessor = RowAccessor::new(&self.row_aggr_schema); - let current = state_buffers - .iter_mut() - .map(|buffer| { - state_accessor.point_to(0, buffer); - acc.evaluate(&state_accessor) - }) - .collect::>>()?; - // Get corresponding field for row accumulator - let field = &output_fields[self.indices[1][idx].start]; - let result = if current.is_empty() { - Ok(arrow::array::new_empty_array(field.data_type())) - } else { - let item = ScalarValue::iter_to_array(current)?; - // cast output if needed (e.g. for types like Dictionary where - // the intermediate GroupByScalar type was not the same as the - // output - cast(&item, field.data_type()) - }?; - results.push(result); - } - results - } - }; + Ok(RecordBatch::try_new(self.schema.clone(), output)?) + } +} - // Store normal accumulator results (either final output or intermediate state): - let mut columns = vec![]; - for (idx, &Range { start, end }) in self.indices[0].iter().enumerate() { - for (field_idx, field) in output_fields[start..end].iter().enumerate() { - let current = match self.mode { - AggregateMode::Partial => ScalarValue::iter_to_array( - group_state_chunk.iter().map(|group_state| { - group_state.accumulator_set[idx] - .state() - .map(|v| v[field_idx].clone()) - .expect("Unexpected accumulator state in hash aggregate") - }), - ), - AggregateMode::Final - | AggregateMode::FinalPartitioned - | AggregateMode::Single => ScalarValue::iter_to_array( - group_state_chunk.iter().map(|group_state| { - group_state.accumulator_set[idx] - .evaluate() - .expect("Unexpected accumulator state in hash aggregate") - }), - ), - }?; - // Cast output if needed (e.g. for types like Dictionary where - // the intermediate GroupByScalar type was not the same as the - // output - let result = cast(¤t, field.data_type())?; - columns.push(result); - } - } +/// Holds structures used for the current input [`RecordBatch`] being +/// processed. Reused across batches here to avoid reallocations +#[derive(Debug, Default)] +struct ScratchSpace { + /// scratch space for the current input [`RecordBatch`] being + /// processed. Reused across batches here to avoid reallocations + current_group_indices: Vec, + // buffer to be reused to store hashes + hashes_buffer: Vec, +} - // Stores the group by fields - let group_buffers = group_state_chunk - .iter() - .map(|gs| gs.group_by_values.row()) - .collect::>(); - let mut output: Vec = self.row_converter.convert_rows(group_buffers)?; +impl ScratchSpace { + fn new() -> Self { + Default::default() + } - // The size of the place occupied by row and normal accumulators - let extra: usize = self - .indices - .iter() - .flatten() - .map(|Range { start, end }| end - start) - .sum(); - let empty_arr = new_null_array(&DataType::Null, 1); - output.extend(std::iter::repeat(empty_arr).take(extra)); - - // Write results of both accumulator types to the corresponding location in - // the output schema: - let results = [columns.into_iter(), row_columns.into_iter()]; - for (outer, mut current) in results.into_iter().enumerate() { - for &Range { start, end } in self.indices[outer].iter() { - for item in output.iter_mut().take(end).skip(start) { - *item = current.next().expect("Columns cannot be empty"); - } - } - } - Ok(Some(RecordBatch::try_new(self.schema.clone(), output)?)) + /// Return the amount of memory alocated by this structure in bytes + fn size(&self) -> usize { + std::mem::size_of_val(self) + + self.current_group_indices.allocated_size() + + self.hashes_buffer.allocated_size() } } diff --git a/datafusion/core/tests/data/empty_0_byte.csv b/datafusion/core/tests/data/empty_0_byte.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/empty_files/all_empty/empty0.csv b/datafusion/core/tests/data/empty_files/all_empty/empty0.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/empty_files/all_empty/empty1.csv b/datafusion/core/tests/data/empty_files/all_empty/empty1.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/empty_files/all_empty/empty2.csv b/datafusion/core/tests/data/empty_files/all_empty/empty2.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/empty_files/some_empty/a_empty.csv b/datafusion/core/tests/data/empty_files/some_empty/a_empty.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/empty_files/some_empty/b.csv b/datafusion/core/tests/data/empty_files/some_empty/b.csv new file mode 100644 index 000000000000..195c0be7c031 --- /dev/null +++ b/datafusion/core/tests/data/empty_files/some_empty/b.csv @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 + diff --git a/datafusion/core/tests/data/empty_files/some_empty/c_empty.csv b/datafusion/core/tests/data/empty_files/some_empty/c_empty.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/empty_files/some_empty/d.csv b/datafusion/core/tests/data/empty_files/some_empty/d.csv new file mode 100644 index 000000000000..195c0be7c031 --- /dev/null +++ b/datafusion/core/tests/data/empty_files/some_empty/d.csv @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 + diff --git a/datafusion/core/tests/data/empty_files/some_empty/e_empty.csv b/datafusion/core/tests/data/empty_files/some_empty/e_empty.csv new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/core/tests/data/one_col.csv b/datafusion/core/tests/data/one_col.csv new file mode 100644 index 000000000000..00a21774480f --- /dev/null +++ b/datafusion/core/tests/data/one_col.csv @@ -0,0 +1,10 @@ +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 diff --git a/datafusion/core/tests/data/wide_rows.csv b/datafusion/core/tests/data/wide_rows.csv new file mode 100644 index 000000000000..22bfb4a0ec9b --- /dev/null +++ b/datafusion/core/tests/data/wide_rows.csv @@ -0,0 +1,3 @@ +1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1 +2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2 + diff --git a/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs b/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs index ed6d1eda17c3..5bb40aca2ab8 100644 --- a/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs +++ b/datafusion/core/tests/sqllogictests/src/engines/datafusion/error.rs @@ -28,45 +28,21 @@ pub type Result = std::result::Result; pub enum DFSqlLogicTestError { /// Error from sqllogictest-rs #[error("SqlLogicTest error(from sqllogictest-rs crate): {0}")] - SqlLogicTest(TestError), + SqlLogicTest(#[from] TestError), /// Error from datafusion #[error("DataFusion error: {0}")] - DataFusion(DataFusionError), + DataFusion(#[from] DataFusionError), /// Error returned when SQL is syntactically incorrect. #[error("SQL Parser error: {0}")] - Sql(ParserError), + Sql(#[from] ParserError), /// Error from arrow-rs #[error("Arrow error: {0}")] - Arrow(ArrowError), + Arrow(#[from] ArrowError), /// Generic error #[error("Other Error: {0}")] Other(String), } -impl From for DFSqlLogicTestError { - fn from(value: TestError) -> Self { - DFSqlLogicTestError::SqlLogicTest(value) - } -} - -impl From for DFSqlLogicTestError { - fn from(value: DataFusionError) -> Self { - DFSqlLogicTestError::DataFusion(value) - } -} - -impl From for DFSqlLogicTestError { - fn from(value: ParserError) -> Self { - DFSqlLogicTestError::Sql(value) - } -} - -impl From for DFSqlLogicTestError { - fn from(value: ArrowError) -> Self { - DFSqlLogicTestError::Arrow(value) - } -} - impl From for DFSqlLogicTestError { fn from(value: String) -> Self { DFSqlLogicTestError::Other(value) diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index bf1ab2cbd1fb..95cf51d57187 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -39,9 +39,10 @@ WITH HEADER ROW LOCATION '../../testing/data/csv/aggregate_test_100.csv' statement ok -CREATE TABLE d_table (c1 decimal(10,3)) as values -(110.000), (110.001), (110.002), (110.003), (110.004), (110.005), (110.006), (110.007), (110.008), (110.009), -(-100.000),(-100.001),(-100.002),(-100.003),(-100.004),(-100.005),(-100.006),(-100.007),(-100.008),(-100.009) +CREATE TABLE d_table (c1 decimal(10,3), c2 varchar) +as values +(110.000, 'A'), (110.001, 'A'), (110.002, 'A'), (110.003, 'A'), (110.004, 'A'), (110.005, 'A'), (110.006, 'A'), (110.007, 'A'), (110.008, 'A'), (110.009, 'A'), +(-100.000, 'B'),(-100.001, 'B'),(-100.002, 'B'),(-100.003, 'B'),(-100.004, 'B'),(-100.005, 'B'),(-100.006, 'B'),(-100.007, 'B'),(-100.008, 'B'),(-100.009, 'B') statement ok CREATE TABLE median_table ( @@ -448,7 +449,7 @@ drop table cpu; # this test is to show create table as and select into works in the same way statement ok -SELECT * INTO cpu +SELECT * INTO cpu FROM (VALUES ('host0', 90.1), ('host1', 90.2), @@ -1483,22 +1484,6 @@ NULL 2 statement ok drop table the_nulls; -# All supported timestamp types - -# "nanos" --> TimestampNanosecondArray -# "micros" --> TimestampMicrosecondArray -# "millis" --> TimestampMillisecondArray -# "secs" --> TimestampSecondArray -# "names" --> StringArray - -statement ok -create table t_source -as values - ('2018-11-13T17:11:10.011375885995', 'Row 0'), - ('2011-12-13T11:13:10.12345', 'Row 1'), - (null, 'Row 2'), - ('2021-01-01T05:11:10.432', 'Row 3'); - statement ok create table bit_aggregate_functions ( c1 SMALLINT NOT NULL, @@ -1568,6 +1553,22 @@ SELECT bool_or(distinct c1), bool_or(distinct c2), bool_or(distinct c3), bool_or ---- true true true false true true false NULL +# All supported timestamp types + +# "nanos" --> TimestampNanosecondArray +# "micros" --> TimestampMicrosecondArray +# "millis" --> TimestampMillisecondArray +# "secs" --> TimestampSecondArray +# "names" --> StringArray + +statement ok +create table t_source +as values + ('2018-11-13T17:11:10.011375885995', 'Row 0', 'X'), + ('2011-12-13T11:13:10.12345', 'Row 1', 'X'), + (null, 'Row 2', 'Y'), + ('2021-01-01T05:11:10.432', 'Row 3', 'Y'); + statement ok create table t as select @@ -1575,29 +1576,38 @@ select arrow_cast(column1, 'Timestamp(Microsecond, None)') as micros, arrow_cast(column1, 'Timestamp(Millisecond, None)') as millis, arrow_cast(column1, 'Timestamp(Second, None)') as secs, - column2 as names + column2 as names, + column3 as tag from t_source; # Demonstate the contents -query PPPPT +query PPPPTT select * from t; ---- -2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 -2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 Row 1 -NULL NULL NULL NULL Row 2 -2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 +2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 X +2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 Row 1 X +NULL NULL NULL NULL Row 2 Y +2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 Y # aggregate_timestamps_sum -statement error Error during planning: The function Sum does not support inputs of type Timestamp\(Nanosecond, None\) +statement error DataFusion error: Error during planning: The function Sum does not support inputs of type Timestamp\(Nanosecond, None\)\. SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t; +statement error DataFusion error: Error during planning: The function Sum does not support inputs of type Timestamp\(Nanosecond, None\)\. +SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag; + # aggregate_timestamps_count query IIII SELECT count(nanos), count(micros), count(millis), count(secs) FROM t; ---- 3 3 3 3 +query TIIII +SELECT tag, count(nanos), count(micros), count(millis), count(secs) FROM t GROUP BY tag ORDER BY tag; +---- +X 2 2 2 2 +Y 1 1 1 1 # aggregate_timestamps_min query PPPP @@ -1605,18 +1615,32 @@ SELECT min(nanos), min(micros), min(millis), min(secs) FROM t; ---- 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 +query TPPPP +SELECT tag, min(nanos), min(micros), min(millis), min(secs) FROM t GROUP BY tag ORDER BY tag; +---- +X 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 +Y 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 + # aggregate_timestamps_max query PPPP SELECT max(nanos), max(micros), max(millis), max(secs) FROM t; ---- 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 +query TPPPP +SELECT tag, max(nanos), max(micros), max(millis), max(secs) FROM t GROUP BY tag ORDER BY tag +---- +X 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 +Y 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 # aggregate_timestamps_avg -statement error Error during planning: The function Avg does not support inputs of type Timestamp\(Nanosecond, None\). +statement error DataFusion error: Error during planning: The function Avg does not support inputs of type Timestamp\(Nanosecond, None\)\. SELECT avg(nanos), avg(micros), avg(millis), avg(secs) FROM t +statement error DataFusion error: Error during planning: The function Avg does not support inputs of type Timestamp\(Nanosecond, None\)\. +SELECT tag, avg(nanos), avg(micros), avg(millis), avg(secs) FROM t GROUP BY tag ORDER BY tag; + statement ok drop table t_source; @@ -1624,6 +1648,101 @@ drop table t_source; statement ok drop table t; + +# All supported Date tpes + +# "date32" --> Date32Array +# "date64" --> Date64Array +# "names" --> StringArray + +statement ok +create table t_source +as values + ('2018-11-13', 'Row 0', 'X'), + ('2011-12-13', 'Row 1', 'X'), + (null, 'Row 2', 'Y'), + ('2021-01-01', 'Row 3', 'Y'); + +statement ok +create table t as +select + arrow_cast(column1, 'Date32') as date32, + -- Workaround https://github.com/apache/arrow-rs/issues/4512 is fixed, can use this + -- arrow_cast(column1, 'Date64') as date64, + arrow_cast(arrow_cast(column1, 'Date32'), 'Date64') as date64, + column2 as names, + column3 as tag +from t_source; + +# Demonstate the contents +query DDTT +select * from t; +---- +2018-11-13 2018-11-13T00:00:00 Row 0 X +2011-12-13 2011-12-13T00:00:00 Row 1 X +NULL NULL Row 2 Y +2021-01-01 2021-01-01T00:00:00 Row 3 Y + + +# aggregate_timestamps_sum +statement error DataFusion error: Error during planning: The function Sum does not support inputs of type Date32\. +SELECT sum(date32), sum(date64) FROM t; + +statement error DataFusion error: Error during planning: The function Sum does not support inputs of type Date32\. +SELECT tag, sum(date32), sum(date64) FROM t GROUP BY tag ORDER BY tag; + +# aggregate_timestamps_count +query II +SELECT count(date32), count(date64) FROM t; +---- +3 3 + +query TII +SELECT tag, count(date32), count(date64) FROM t GROUP BY tag ORDER BY tag; +---- +X 2 2 +Y 1 1 + +# aggregate_timestamps_min +query DD +SELECT min(date32), min(date64) FROM t; +---- +2011-12-13 2011-12-13T00:00:00 + +query TDD +SELECT tag, min(date32), min(date64) FROM t GROUP BY tag ORDER BY tag; +---- +X 2011-12-13 2011-12-13T00:00:00 +Y 2021-01-01 2021-01-01T00:00:00 + +# aggregate_timestamps_max +query DD +SELECT max(date32), max(date64) FROM t; +---- +2021-01-01 2021-01-01T00:00:00 + +query TDD +SELECT tag, max(date32), max(date64) FROM t GROUP BY tag ORDER BY tag +---- +X 2018-11-13 2018-11-13T00:00:00 +Y 2021-01-01 2021-01-01T00:00:00 + + +# aggregate_timestamps_avg +statement error DataFusion error: Error during planning: The function Avg does not support inputs of type Date32\. +SELECT avg(date32), avg(date64) FROM t + +statement error DataFusion error: Error during planning: The function Avg does not support inputs of type Date32\. +SELECT tag, avg(date32), avg(date64) FROM t GROUP BY tag ORDER BY tag; + + +statement ok +drop table t_source; + +statement ok +drop table t; + + # All supported time types # Columns are named: @@ -1636,10 +1755,10 @@ drop table t; statement ok create table t_source as values - ('18:06:30.243620451', 'Row 0'), - ('20:08:28.161121654', 'Row 1'), - ('19:11:04.156423842', 'Row 2'), - ('21:06:28.247821084', 'Row 3'); + ('18:06:30.243620451', 'Row 0', 'A'), + ('20:08:28.161121654', 'Row 1', 'A'), + ('19:11:04.156423842', 'Row 2', 'B'), + ('21:06:28.247821084', 'Row 3', 'B'); statement ok @@ -1649,28 +1768,38 @@ select arrow_cast(column1, 'Time64(Microsecond)') as micros, arrow_cast(column1, 'Time32(Millisecond)') as millis, arrow_cast(column1, 'Time32(Second)') as secs, - column2 as names + column2 as names, + column3 as tag from t_source; # Demonstate the contents -query DDDDT +query DDDDTT select * from t; ---- -18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 Row 0 -20:08:28.161121654 20:08:28.161121 20:08:28.161 20:08:28 Row 1 -19:11:04.156423842 19:11:04.156423 19:11:04.156 19:11:04 Row 2 -21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 Row 3 +18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 Row 0 A +20:08:28.161121654 20:08:28.161121 20:08:28.161 20:08:28 Row 1 A +19:11:04.156423842 19:11:04.156423 19:11:04.156 19:11:04 Row 2 B +21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 Row 3 B # aggregate_times_sum statement error DataFusion error: Error during planning: The function Sum does not support inputs of type Time64\(Nanosecond\). SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t +statement error DataFusion error: Error during planning: The function Sum does not support inputs of type Time64\(Nanosecond\)\. +SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag + # aggregate_times_count query IIII SELECT count(nanos), count(micros), count(millis), count(secs) FROM t ---- 4 4 4 4 +query TIIII +SELECT tag, count(nanos), count(micros), count(millis), count(secs) FROM t GROUP BY tag ORDER BY tag +---- +A 2 2 2 2 +B 2 2 2 2 + # aggregate_times_min query DDDD @@ -1678,17 +1807,32 @@ SELECT min(nanos), min(micros), min(millis), min(secs) FROM t ---- 18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 +query TDDDD +SELECT tag, min(nanos), min(micros), min(millis), min(secs) FROM t GROUP BY tag ORDER BY tag +---- +A 18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 +B 19:11:04.156423842 19:11:04.156423 19:11:04.156 19:11:04 + # aggregate_times_max query DDDD SELECT max(nanos), max(micros), max(millis), max(secs) FROM t ---- 21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 +query TDDDD +SELECT tag, max(nanos), max(micros), max(millis), max(secs) FROM t GROUP BY tag ORDER BY tag +---- +A 20:08:28.161121654 20:08:28.161121 20:08:28.161 20:08:28 +B 21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 + # aggregate_times_avg statement error DataFusion error: Error during planning: The function Avg does not support inputs of type Time64\(Nanosecond\). SELECT avg(nanos), avg(micros), avg(millis), avg(secs) FROM t +statement error DataFusion error: Error during planning: The function Avg does not support inputs of type Time64\(Nanosecond\)\. +SELECT tag, avg(nanos), avg(micros), avg(millis), avg(secs) FROM t GROUP BY tag ORDER BY tag; + statement ok drop table t_source; @@ -1710,6 +1854,12 @@ select sum(c1), arrow_typeof(sum(c1)) from d_table; ---- 100 Decimal128(20, 3) +query TRT +select c2, sum(c1), arrow_typeof(sum(c1)) from d_table GROUP BY c2 ORDER BY c2; +---- +A 1100.045 Decimal128(20, 3) +B -1000.045 Decimal128(20, 3) + # aggregate_decimal_avg query RT @@ -1717,6 +1867,12 @@ select avg(c1), arrow_typeof(avg(c1)) from d_table ---- 5 Decimal128(14, 7) +query TRT +select c2, avg(c1), arrow_typeof(avg(c1)) from d_table GROUP BY c2 ORDER BY c2 +---- +A 110.0045 Decimal128(14, 7) +B -100.0045 Decimal128(14, 7) + # Use PostgresSQL dialect statement ok set datafusion.sql_parser.dialect = 'Postgres'; diff --git a/datafusion/core/tests/sqllogictests/test_files/insert.slt b/datafusion/core/tests/sqllogictests/test_files/insert.slt index 9f4122ac5ba9..faa519834c6f 100644 --- a/datafusion/core/tests/sqllogictests/test_files/insert.slt +++ b/datafusion/core/tests/sqllogictests/test_files/insert.slt @@ -299,4 +299,4 @@ select * from table_without_values; 2 NULL statement ok -drop table table_without_values; +drop table table_without_values; \ No newline at end of file diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part index 776c13e928fe..050f14aaede5 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part @@ -57,9 +57,7 @@ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS --------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------FilterExec: l_shipdate@6 <= 10471 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false - +--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false query TTRRRRRRRI select diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part index 6c662c1091b9..d46536a25349 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part @@ -100,15 +100,13 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 -----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +----------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] --------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------FilterExec: l_returnflag@3 = R -------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false +------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 --------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part index 0c16fe1ab98d..9118935c4b73 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part @@ -93,8 +93,7 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false +--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 --------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -120,8 +119,7 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false +--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 --------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part index dda4ff6fd7a0..66ff3f8964d6 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part @@ -74,12 +74,10 @@ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] ------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_commitdate@2 < l_receiptdate@3 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 8766 AND l_receiptdate@3 < 9131 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false +------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderpriority], has_header=false +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part index 8ac9576a1248..bd358962b5e1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part @@ -78,8 +78,7 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part index bff3ccf4fac6..a22b1206ca4f 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part @@ -53,8 +53,7 @@ ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") ------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part index 700ecad7c66c..613f44f2970d 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part @@ -94,8 +94,7 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([MAX(total_revenue)@0], 4), input_partitions=1 --------------AggregateExec: mode=Final, gby=[], aggr=[MAX(total_revenue)] @@ -110,8 +109,7 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] --------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false query ITTTR diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part index 58796e93a893..1f24791a5693 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part @@ -87,8 +87,7 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)] --------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey], has_header=false +------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey], has_header=false --------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part index ca90c78280d4..21a13e65dce5 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part @@ -65,8 +65,7 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av ----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false +----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] @@ -79,8 +78,7 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av ------------------CoalesceBatchesExec: target_batch_size=8192 --------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ----------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_quantity], has_header=false +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part index 3570b7bd8939..11a188301dd7 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part @@ -89,12 +89,10 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] --------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_quantity], has_header=false +--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false ----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] ------------------CoalesceBatchesExec: target_batch_size=8192 --------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 @@ -102,8 +100,7 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_quantity], has_header=false +------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part index 0906aa4aad2b..c5c2836b76b1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part @@ -76,8 +76,7 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco ------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part index 18cd261b7663..3ad63f482a3a 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part @@ -135,8 +135,7 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -174,8 +173,7 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 --------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part index ec9bb4ee4140..16ef6ce83b7a 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part @@ -113,8 +113,7 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] --------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false +--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 --------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] @@ -130,8 +129,7 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part index 04c45ec5bbd3..a43c761d88c5 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part @@ -126,15 +126,13 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ----------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +----------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false --------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------FilterExec: o_orderstatus@1 = F -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderstatus], has_header=false +------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 --------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] @@ -144,15 +142,13 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey], has_header=false +----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part index e5cbc028220a..3fb1a1d1fdca 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part @@ -94,8 +94,7 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_custkey], has_header=false +----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], has_header=false ------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] --------------------CoalescePartitionsExec ----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part index f8c13856812a..91af8b77996d 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part @@ -86,15 +86,13 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------FilterExec: o_orderdate@2 < 9204 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false +----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 --------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------FilterExec: l_shipdate@3 > 9204 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false +--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part index d3e6afa406f1..e4660cf90420 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part @@ -68,15 +68,13 @@ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] ------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false +------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------FilterExec: l_commitdate@1 < l_receiptdate@2 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false +------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part index 898f21d3d52c..c7e8607e0398 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part @@ -106,12 +106,10 @@ SortPreservingMergeExec: [revenue@1 DESC] ----------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 -----------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +----------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false +------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false ----------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 --------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q6.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q6.slt.part index e388f800725b..9acaee2aa61f 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q6.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q6.slt.part @@ -41,8 +41,7 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as r --------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------CoalesceBatchesExec: target_batch_size=8192 ------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false +--------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part index 741fc1864e5a..991499813c00 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part @@ -122,12 +122,10 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L --------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 ---------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +--------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey], has_header=false +------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], has_header=false ----------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 --------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part index 15137267dc43..96f13b2a341d 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part @@ -139,8 +139,7 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] --------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false --------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +------------------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false ----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 --------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -149,8 +148,7 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] --------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 ---------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +--------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part index 45a4be6466e4..2feaef32cf4b 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part @@ -116,20 +116,17 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false --------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 --------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderdate], has_header=false +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 --------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index c4d745f8f190..09339d7e499f 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -448,7 +448,7 @@ ORDER BY c9 LIMIT 5 ---- -48302 -16100.666666666666 3 -11243 3747.666666666666 3 +11243 3747.666666666667 3 -51311 -17103.666666666668 3 -2391 -797 3 46756 15585.333333333334 3 @@ -468,7 +468,7 @@ LIMIT 5 46721.33333333174 31147.555555554496 216.151181660734 176.486700789477 2639429.333333332 1759619.5555555548 1624.632060908971 1326.50652299774 746202.3333333324 497468.2222222216 863.830037295146 705.314271954156 -768422.9999999981 512281.9999999988 876.597399037892 715.738779164577 +768422.9999999981 512281.9999999988 876.597399037893 715.738779164577 66526.3333333288 44350.88888888587 257.926992254259 210.596507304575 # window_frame_rows_preceding_with_partition_unique_order_by diff --git a/datafusion/core/tests/user_defined/user_defined_window_functions.rs b/datafusion/core/tests/user_defined/user_defined_window_functions.rs index 8736ede690f1..1331347fac80 100644 --- a/datafusion/core/tests/user_defined/user_defined_window_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_window_functions.rs @@ -515,7 +515,7 @@ impl PartitionEvaluator for OddCounter { println!("evaluate, values: {values:#?}, range: {range:?}"); self.test_state.inc_evaluate_called(); - let values: &Int64Array = values.get(0).unwrap().as_primitive(); + let values: &Int64Array = values[0].as_primitive(); let values = values.slice(range.start, range.len()); let scalar = ScalarValue::Int64( match (odd_count(&values), self.test_state.null_for_zero) { @@ -534,10 +534,7 @@ impl PartitionEvaluator for OddCounter { println!("evaluate_all, values: {values:#?}, num_rows: {num_rows}"); self.test_state.inc_evaluate_all_called(); - Ok(odd_count_arr( - values.get(0).unwrap().as_primitive(), - num_rows, - )) + Ok(odd_count_arr(values[0].as_primitive(), num_rows)) } fn evaluate_all_with_rank( diff --git a/datafusion/execution/src/memory_pool/proxy.rs b/datafusion/execution/src/memory_pool/proxy.rs index 43532f9a81f1..2bf485c6ee76 100644 --- a/datafusion/execution/src/memory_pool/proxy.rs +++ b/datafusion/execution/src/memory_pool/proxy.rs @@ -26,6 +26,11 @@ pub trait VecAllocExt { /// [Push](Vec::push) new element to vector and store additional allocated bytes in `accounting` (additive). fn push_accounted(&mut self, x: Self::T, accounting: &mut usize); + + /// Return the amount of memory allocated by this Vec (not + /// recursively counting any heap allocations contained within the + /// structure). Does not include the size of `self` + fn allocated_size(&self) -> usize; } impl VecAllocExt for Vec { @@ -44,6 +49,9 @@ impl VecAllocExt for Vec { self.push(x); } + fn allocated_size(&self) -> usize { + std::mem::size_of::() * self.capacity() + } } /// Extension trait for [`RawTable`] to account for allocations. diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index d1c2f7bf3377..b7ffa1810cce 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -62,6 +62,7 @@ indexmap = "2.0.0" itertools = { version = "0.11", features = ["use_std"] } lazy_static = { version = "^1.4.0" } libc = "0.2.140" +log = "^0.4" md-5 = { version = "^0.10.0", optional = true } paste = "^1.0" petgraph = "0.6.2" diff --git a/datafusion/physical-expr/src/aggregate/average.rs b/datafusion/physical-expr/src/aggregate/average.rs index 3c76da51a9d4..e95e9fcf877a 100644 --- a/datafusion/physical-expr/src/aggregate/average.rs +++ b/datafusion/physical-expr/src/aggregate/average.rs @@ -17,10 +17,14 @@ //! Defines physical expressions that can evaluated at runtime during query execution +use arrow::array::{AsArray, PrimitiveBuilder}; +use log::debug; + use std::any::Any; use std::convert::TryFrom; use std::sync::Arc; +use crate::aggregate::groups_accumulator::accumulate::NullState; use crate::aggregate::row_accumulator::{ is_row_accumulator_support_dtype, RowAccumulator, }; @@ -29,19 +33,23 @@ use crate::aggregate::sum::sum_batch; use crate::aggregate::utils::calculate_result_decimal_for_avg; use crate::aggregate::utils::down_cast_any_ref; use crate::expressions::format_state_name; -use crate::{AggregateExpr, PhysicalExpr}; +use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr}; use arrow::compute; -use arrow::datatypes::DataType; +use arrow::datatypes::{DataType, Decimal128Type, Float64Type, UInt64Type}; use arrow::{ array::{ArrayRef, UInt64Array}, datatypes::Field, }; -use arrow_array::Array; +use arrow_array::{ + Array, ArrowNativeTypeOp, ArrowNumericType, ArrowPrimitiveType, PrimitiveArray, +}; use datafusion_common::{downcast_value, ScalarValue}; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::Accumulator; use datafusion_row::accessor::RowAccessor; +use super::utils::{adjust_output_array, Decimal128Averager}; + /// AVG aggregate expression #[derive(Debug, Clone)] pub struct Avg { @@ -155,6 +163,50 @@ impl AggregateExpr for Avg { &self.rt_data_type, )?)) } + + fn groups_accumulator_supported(&self) -> bool { + use DataType::*; + + matches!(&self.rt_data_type, Float64 | Decimal128(_, _)) + } + + fn create_groups_accumulator(&self) -> Result> { + use DataType::*; + // instantiate specialized accumulator based for the type + match (&self.sum_data_type, &self.rt_data_type) { + (Float64, Float64) => { + Ok(Box::new(AvgGroupsAccumulator::::new( + &self.sum_data_type, + &self.rt_data_type, + |sum: f64, count: u64| Ok(sum / count as f64), + ))) + } + ( + Decimal128(_sum_precision, sum_scale), + Decimal128(target_precision, target_scale), + ) => { + let decimal_averager = Decimal128Averager::try_new( + *sum_scale, + *target_precision, + *target_scale, + )?; + + let avg_fn = + move |sum: i128, count: u64| decimal_averager.avg(sum, count as i128); + + Ok(Box::new(AvgGroupsAccumulator::::new( + &self.sum_data_type, + &self.rt_data_type, + avg_fn, + ))) + } + + _ => Err(DataFusionError::NotImplemented(format!( + "AvgGroupsAccumulator for ({} --> {})", + self.sum_data_type, self.rt_data_type, + ))), + } + } } impl PartialEq for Avg { @@ -383,6 +435,190 @@ impl RowAccumulator for AvgRowAccumulator { } } +/// An accumulator to compute the average of `[PrimitiveArray]`. +/// Stores values as native types, and does overflow checking +/// +/// F: Function that calculates the average value from a sum of +/// T::Native and a total count +#[derive(Debug)] +struct AvgGroupsAccumulator +where + T: ArrowNumericType + Send, + F: Fn(T::Native, u64) -> Result + Send, +{ + /// The type of the internal sum + sum_data_type: DataType, + + /// The type of the returned sum + return_data_type: DataType, + + /// Count per group (use u64 to make UInt64Array) + counts: Vec, + + /// Sums per group, stored as the native type + sums: Vec, + + /// Track nulls in the input / filters + null_state: NullState, + + /// Function that computes the final average (value / count) + avg_fn: F, +} + +impl AvgGroupsAccumulator +where + T: ArrowNumericType + Send, + F: Fn(T::Native, u64) -> Result + Send, +{ + pub fn new(sum_data_type: &DataType, return_data_type: &DataType, avg_fn: F) -> Self { + debug!( + "AvgGroupsAccumulator ({}, sum type: {sum_data_type:?}) --> {return_data_type:?}", + std::any::type_name::() + ); + + Self { + return_data_type: return_data_type.clone(), + sum_data_type: sum_data_type.clone(), + counts: vec![], + sums: vec![], + null_state: NullState::new(), + avg_fn, + } + } +} + +impl GroupsAccumulator for AvgGroupsAccumulator +where + T: ArrowNumericType + Send, + F: Fn(T::Native, u64) -> Result + Send, +{ + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + + // increment counts, update sums + self.counts.resize(total_num_groups, 0); + self.sums.resize(total_num_groups, T::default_value()); + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let sum = &mut self.sums[group_index]; + *sum = sum.add_wrapping(new_value); + + self.counts[group_index] += 1; + }, + ); + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + // first batch is counts, second is partial sums + let partial_counts = values[0].as_primitive::(); + let partial_sums = values[1].as_primitive::(); + // update counts with partial counts + self.counts.resize(total_num_groups, 0); + self.null_state.accumulate( + group_indices, + partial_counts, + opt_filter, + total_num_groups, + |group_index, partial_count| { + self.counts[group_index] += partial_count; + }, + ); + + // update sums + self.sums.resize(total_num_groups, T::default_value()); + self.null_state.accumulate( + group_indices, + partial_sums, + opt_filter, + total_num_groups, + |group_index, new_value: ::Native| { + let sum = &mut self.sums[group_index]; + *sum = sum.add_wrapping(new_value); + }, + ); + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let counts = std::mem::take(&mut self.counts); + let sums = std::mem::take(&mut self.sums); + let nulls = self.null_state.build(); + + assert_eq!(nulls.len(), sums.len()); + assert_eq!(counts.len(), sums.len()); + + // don't evaluate averages with null inputs to avoid errors on null values + + let array: PrimitiveArray = if nulls.null_count() > 0 { + let mut builder = PrimitiveBuilder::::with_capacity(nulls.len()); + let iter = sums.into_iter().zip(counts.into_iter()).zip(nulls.iter()); + + for ((sum, count), is_valid) in iter { + if is_valid { + builder.append_value((self.avg_fn)(sum, count)?) + } else { + builder.append_null(); + } + } + builder.finish() + } else { + let averages: Vec = sums + .into_iter() + .zip(counts.into_iter()) + .map(|(sum, count)| (self.avg_fn)(sum, count)) + .collect::>>()?; + PrimitiveArray::new(averages.into(), Some(nulls)) // no copy + }; + + // fix up decimal precision and scale for decimals + let array = adjust_output_array(&self.return_data_type, Arc::new(array))?; + + Ok(array) + } + + // return arrays for sums and counts + fn state(&mut self) -> Result> { + let nulls = Some(self.null_state.build()); + let counts = std::mem::take(&mut self.counts); + let counts = UInt64Array::new(counts.into(), nulls.clone()); // zero copy + + let sums = std::mem::take(&mut self.sums); + let sums = PrimitiveArray::::new(sums.into(), nulls); // zero copy + let sums = adjust_output_array(&self.sum_data_type, Arc::new(sums))?; + + Ok(vec![ + Arc::new(counts) as ArrayRef, + Arc::new(sums) as ArrayRef, + ]) + } + + fn size(&self) -> usize { + self.counts.capacity() * std::mem::size_of::() + + self.sums.capacity() * std::mem::size_of::() + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs index 4bbe563edce8..ab37e5891e3f 100644 --- a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs +++ b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs @@ -15,15 +15,18 @@ // specific language governing permissions and limitations // under the License. -//! Defines physical expressions that can evaluated at runtime during query execution +//! Defines BitAnd, BitOr, and BitXor Aggregate accumulators use ahash::RandomState; use std::any::Any; use std::convert::TryFrom; use std::sync::Arc; -use crate::{AggregateExpr, PhysicalExpr}; -use arrow::datatypes::DataType; +use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr}; +use arrow::datatypes::{ + DataType, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, + UInt64Type, UInt8Type, +}; use arrow::{ array::{ ArrayRef, Int16Array, Int32Array, Int64Array, Int8Array, UInt16Array, @@ -35,6 +38,7 @@ use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; use std::collections::HashSet; +use crate::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; use crate::aggregate::row_accumulator::{ is_row_accumulator_support_dtype, RowAccumulator, }; @@ -44,6 +48,19 @@ use arrow::array::Array; use arrow::compute::{bit_and, bit_or, bit_xor}; use datafusion_row::accessor::RowAccessor; +/// Creates a [`PrimitiveGroupsAccumulator`] with the specified +/// [`ArrowPrimitiveType`] which applies `$FN` to each element +/// +/// [`ArrowPrimitiveType`]: arrow::datatypes::ArrowPrimitiveType +macro_rules! instantiate_primitive_accumulator { + ($SELF:expr, $PRIMTYPE:ident, $FN:expr) => {{ + Ok(Box::new(PrimitiveGroupsAccumulator::<$PRIMTYPE, _>::new( + &$SELF.data_type, + $FN, + ))) + }}; +} + // returns the new value after bit_and/bit_or/bit_xor with the new values, taking nullability into account macro_rules! typed_bit_and_or_xor_batch { ($VALUES:expr, $ARRAYTYPE:ident, $SCALAR:ident, $OP:ident) => {{ @@ -254,6 +271,54 @@ impl AggregateExpr for BitAnd { ))) } + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result> { + use std::ops::BitAndAssign; + match self.data_type { + DataType::Int8 => { + instantiate_primitive_accumulator!(self, Int8Type, |x, y| x + .bitand_assign(y)) + } + DataType::Int16 => { + instantiate_primitive_accumulator!(self, Int16Type, |x, y| x + .bitand_assign(y)) + } + DataType::Int32 => { + instantiate_primitive_accumulator!(self, Int32Type, |x, y| x + .bitand_assign(y)) + } + DataType::Int64 => { + instantiate_primitive_accumulator!(self, Int64Type, |x, y| x + .bitand_assign(y)) + } + DataType::UInt8 => { + instantiate_primitive_accumulator!(self, UInt8Type, |x, y| x + .bitand_assign(y)) + } + DataType::UInt16 => { + instantiate_primitive_accumulator!(self, UInt16Type, |x, y| x + .bitand_assign(y)) + } + DataType::UInt32 => { + instantiate_primitive_accumulator!(self, UInt32Type, |x, y| x + .bitand_assign(y)) + } + DataType::UInt64 => { + instantiate_primitive_accumulator!(self, UInt64Type, |x, y| x + .bitand_assign(y)) + } + + _ => Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator not supported for {} with {}", + self.name(), + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } @@ -444,6 +509,54 @@ impl AggregateExpr for BitOr { ))) } + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result> { + use std::ops::BitOrAssign; + match self.data_type { + DataType::Int8 => { + instantiate_primitive_accumulator!(self, Int8Type, |x, y| x + .bitor_assign(y)) + } + DataType::Int16 => { + instantiate_primitive_accumulator!(self, Int16Type, |x, y| x + .bitor_assign(y)) + } + DataType::Int32 => { + instantiate_primitive_accumulator!(self, Int32Type, |x, y| x + .bitor_assign(y)) + } + DataType::Int64 => { + instantiate_primitive_accumulator!(self, Int64Type, |x, y| x + .bitor_assign(y)) + } + DataType::UInt8 => { + instantiate_primitive_accumulator!(self, UInt8Type, |x, y| x + .bitor_assign(y)) + } + DataType::UInt16 => { + instantiate_primitive_accumulator!(self, UInt16Type, |x, y| x + .bitor_assign(y)) + } + DataType::UInt32 => { + instantiate_primitive_accumulator!(self, UInt32Type, |x, y| x + .bitor_assign(y)) + } + DataType::UInt64 => { + instantiate_primitive_accumulator!(self, UInt64Type, |x, y| x + .bitor_assign(y)) + } + + _ => Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator not supported for {} with {}", + self.name(), + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } @@ -635,6 +748,54 @@ impl AggregateExpr for BitXor { ))) } + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result> { + use std::ops::BitXorAssign; + match self.data_type { + DataType::Int8 => { + instantiate_primitive_accumulator!(self, Int8Type, |x, y| x + .bitxor_assign(y)) + } + DataType::Int16 => { + instantiate_primitive_accumulator!(self, Int16Type, |x, y| x + .bitxor_assign(y)) + } + DataType::Int32 => { + instantiate_primitive_accumulator!(self, Int32Type, |x, y| x + .bitxor_assign(y)) + } + DataType::Int64 => { + instantiate_primitive_accumulator!(self, Int64Type, |x, y| x + .bitxor_assign(y)) + } + DataType::UInt8 => { + instantiate_primitive_accumulator!(self, UInt8Type, |x, y| x + .bitxor_assign(y)) + } + DataType::UInt16 => { + instantiate_primitive_accumulator!(self, UInt16Type, |x, y| x + .bitxor_assign(y)) + } + DataType::UInt32 => { + instantiate_primitive_accumulator!(self, UInt32Type, |x, y| x + .bitxor_assign(y)) + } + DataType::UInt64 => { + instantiate_primitive_accumulator!(self, UInt64Type, |x, y| x + .bitxor_assign(y)) + } + + _ => Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator not supported for {} with {}", + self.name(), + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } diff --git a/datafusion/physical-expr/src/aggregate/bool_and_or.rs b/datafusion/physical-expr/src/aggregate/bool_and_or.rs index e444dc61ee1b..6107b0972c81 100644 --- a/datafusion/physical-expr/src/aggregate/bool_and_or.rs +++ b/datafusion/physical-expr/src/aggregate/bool_and_or.rs @@ -17,10 +17,7 @@ //! Defines physical expressions that can evaluated at runtime during query execution -use std::any::Any; -use std::sync::Arc; - -use crate::{AggregateExpr, PhysicalExpr}; +use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr}; use arrow::datatypes::DataType; use arrow::{ array::{ArrayRef, BooleanArray}, @@ -28,7 +25,10 @@ use arrow::{ }; use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; +use std::any::Any; +use std::sync::Arc; +use crate::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; use crate::aggregate::row_accumulator::{ is_row_accumulator_support_dtype, RowAccumulator, }; @@ -193,6 +193,23 @@ impl AggregateExpr for BoolAnd { ))) } + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result> { + match self.data_type { + DataType::Boolean => { + Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x && y))) + } + _ => Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator not supported for {} with {}", + self.name(), + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } @@ -381,6 +398,23 @@ impl AggregateExpr for BoolOr { ))) } + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result> { + match self.data_type { + DataType::Boolean => { + Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x || y))) + } + _ => Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator not supported for {} with {}", + self.name(), + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } diff --git a/datafusion/physical-expr/src/aggregate/count.rs b/datafusion/physical-expr/src/aggregate/count.rs index 22cb2512fc42..e0b9ffd81ae5 100644 --- a/datafusion/physical-expr/src/aggregate/count.rs +++ b/datafusion/physical-expr/src/aggregate/count.rs @@ -24,11 +24,14 @@ use std::sync::Arc; use crate::aggregate::row_accumulator::RowAccumulator; use crate::aggregate::utils::down_cast_any_ref; -use crate::{AggregateExpr, PhysicalExpr}; +use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr}; use arrow::array::{Array, Int64Array}; use arrow::compute; use arrow::datatypes::DataType; use arrow::{array::ArrayRef, datatypes::Field}; +use arrow_array::cast::AsArray; +use arrow_array::types::Int64Type; +use arrow_array::PrimitiveArray; use arrow_buffer::BooleanBuffer; use datafusion_common::{downcast_value, ScalarValue}; use datafusion_common::{DataFusionError, Result}; @@ -37,6 +40,8 @@ use datafusion_row::accessor::RowAccessor; use crate::expressions::format_state_name; +use super::groups_accumulator::accumulate::accumulate_indices; + /// COUNT aggregate expression /// Returns the amount of non-null values of the given expression. #[derive(Debug, Clone)] @@ -44,6 +49,10 @@ pub struct Count { name: String, data_type: DataType, nullable: bool, + /// Input exprs + /// + /// For `COUNT(c1)` this is `[c1]` + /// For `COUNT(c1, c2)` this is `[c1, c2]` exprs: Vec>, } @@ -76,6 +85,114 @@ impl Count { } } +/// An accumulator to compute the counts of [`PrimitiveArray`]. +/// Stores values as native types, and does overflow checking +/// +/// Unlike most other accumulators, COUNT never produces NULLs. If no +/// non-null values are seen in any group the output is 0. Thus, this +/// accumulator has no additional null or seen filter tracking. +#[derive(Debug)] +struct CountGroupsAccumulator { + /// Count per group. + /// + /// Note this is an i64 and not a u64 (or usize) because the + /// output type of count is `DataType::Int64`. Thus by using `i64` + /// for the counts, the output [`Int64Array`] can be created + /// without copy. + counts: Vec, +} + +impl CountGroupsAccumulator { + pub fn new() -> Self { + Self { counts: vec![] } + } +} + +impl GroupsAccumulator for CountGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = &values[0]; + + // Add one to each group's counter for each non null, non + // filtered value + self.counts.resize(total_num_groups, 0); + accumulate_indices( + group_indices, + values.nulls(), // ignore values + opt_filter, + |group_index| { + self.counts[group_index] += 1; + }, + ); + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "one argument to merge_batch"); + // first batch is counts, second is partial sums + let partial_counts = values[0].as_primitive::(); + + // intermediate counts are always created as non null + assert_eq!(partial_counts.null_count(), 0); + let partial_counts = partial_counts.values(); + + // Adds the counts with the partial counts + self.counts.resize(total_num_groups, 0); + match opt_filter { + Some(filter) => filter + .iter() + .zip(group_indices.iter()) + .zip(partial_counts.iter()) + .for_each(|((filter_value, &group_index), partial_count)| { + if let Some(true) = filter_value { + self.counts[group_index] += partial_count; + } + }), + None => group_indices.iter().zip(partial_counts.iter()).for_each( + |(&group_index, partial_count)| { + self.counts[group_index] += partial_count; + }, + ), + } + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let counts = std::mem::take(&mut self.counts); + + // Count is always non null (null inputs just don't contribute to the overall values) + let nulls = None; + let array = PrimitiveArray::::new(counts.into(), nulls); + + Ok(Arc::new(array)) + } + + // return arrays for counts + fn state(&mut self) -> Result> { + let counts = std::mem::take(&mut self.counts); + let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls + Ok(vec![Arc::new(counts) as ArrayRef]) + } + + fn size(&self) -> usize { + self.counts.capacity() * std::mem::size_of::() + } +} + /// count null values for multiple columns /// for each row if one column value is null, then null_count + 1 fn null_count_for_multiple_cols(values: &[ArrayRef]) -> usize { @@ -133,6 +250,12 @@ impl AggregateExpr for Count { true } + fn groups_accumulator_supported(&self) -> bool { + // groups accumulator only supports `COUNT(c1)`, not + // `COUNT(c1, c2)`, etc + self.exprs.len() == 1 + } + fn create_row_accumulator( &self, start_index: usize, @@ -147,6 +270,11 @@ impl AggregateExpr for Count { fn create_sliding_accumulator(&self) -> Result> { Ok(Box::new(CountAccumulator::new())) } + + fn create_groups_accumulator(&self) -> Result> { + // instantiate specialized accumulator + Ok(Box::new(CountGroupsAccumulator::new())) + } } impl PartialEq for Count { diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs new file mode 100644 index 000000000000..bcc9d30bedd8 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs @@ -0,0 +1,854 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! [`GroupsAccumulator`] helpers: [`NullState`] and [`accumulate_indices`] +//! +//! [`GroupsAccumulator`]: crate::GroupsAccumulator + +use arrow::datatypes::ArrowPrimitiveType; +use arrow_array::{Array, BooleanArray, PrimitiveArray}; +use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; + +/// Track the accumulator null state per row: if any values for that +/// group were null and if any values have been seen at all for that group. +/// +/// This is part of the inner loop for many [`GroupsAccumulator`]s, +/// and thus the performance is critical and so there are multiple +/// specialized implementations, invoked depending on the specific +/// combinations of the input. +/// +/// Typically there are 4 potential combinations of inputs must be +/// special cased for performance: +/// +/// * With / Without filter +/// * With / Without nulls in the input +/// +/// If the input has nulls, then the accumulator must potentially +/// handle each input null value specially (e.g. for `SUM` to mark the +/// corresponding sum as null) +/// +/// If there are filters present, `NullState` tracks if it has seen +/// *any* value for that group (as some values may be filtered +/// out). Without a filter, the accumulator is only passed groups that +/// had at least one value to accumulate so they do not need to track +/// if they have seen values for a particular group. +/// +/// [`GroupsAccumulator`]: crate::GroupsAccumulator +#[derive(Debug)] +pub struct NullState { + /// Have we seen any non-filtered input values for `group_index`? + /// + /// If `seen_values[i]` is true, have seen at least one non null + /// value for group `i` + /// + /// If `seen_values[i]` is false, have not seen any values that + /// pass the filter yet for group `i` + seen_values: BooleanBufferBuilder, +} + +impl NullState { + pub fn new() -> Self { + Self { + seen_values: BooleanBufferBuilder::new(0), + } + } + + /// return the size of all buffers allocated by this null state, not including self + pub fn size(&self) -> usize { + // capacity is in bits, so convert to bytes + self.seen_values.capacity() / 8 + } + + /// Invokes `value_fn(group_index, value)` for each non null, non + /// filtered value of `value`, while tracking which groups have + /// seen null inputs and which groups have seen any inputs if necessary + // + /// # Arguments: + /// + /// * `values`: the input arguments to the accumulator + /// * `group_indices`: To which groups do the rows in `values` belong, (aka group_index) + /// * `opt_filter`: if present, only rows for which is Some(true) are included + /// * `value_fn`: function invoked for (group_index, value) where value is non null + /// + /// # Example + /// + /// ```text + /// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ + /// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ + /// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ + /// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ + /// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ + /// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ + /// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ + /// │ └─────┘ │ │ └─────┘ │ └─────┘ + /// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ + /// + /// group_indices values opt_filter + /// ``` + /// + /// In the example above, `value_fn` is invoked for each (group_index, + /// value) pair where `opt_filter[i]` is true and values is non null + /// + /// ```text + /// value_fn(2, 200) + /// value_fn(0, 200) + /// value_fn(0, 300) + /// ``` + /// + /// It also sets + /// + /// 1. `self.seen_values[group_index]` to true for all rows that had a non null vale + pub fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + mut value_fn: F, + ) where + T: ArrowPrimitiveType + Send, + F: FnMut(usize, T::Native) + Send, + { + let data: &[T::Native] = values.values(); + assert_eq!(data.len(), group_indices.len()); + + // ensure the seen_values is big enough (start everything at + // "not seen" valid) + let seen_values = + initialize_builder(&mut self.seen_values, total_num_groups, false); + + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + let iter = group_indices.iter().zip(data.iter()); + for (&group_index, &new_value) in iter { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value); + } + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let data_chunks = data.chunks_exact(64); + let bit_chunks = nulls.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + let data_remainder = data_chunks.remainder(); + + group_indices_chunks + .zip(data_chunks) + .zip(bit_chunks.iter()) + .for_each(|((group_index_chunk, data_chunk), mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().zip(data_chunk.iter()).for_each( + |(&group_index, &new_value)| { + // valid bit was set, real value + let is_valid = (mask & index_mask) != 0; + if is_valid { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value); + } + index_mask <<= 1; + }, + ) + }); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .zip(data_remainder.iter()) + .enumerate() + .for_each(|(i, (&group_index, &new_value))| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value); + } + }); + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, &new_value), filter_value)| { + if let Some(true) = filter_value { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value) + } + } + }) + } + } + } + + /// Invokes `value_fn(group_index, value)` for each non null, non + /// filtered value in `values`, while tracking which groups have + /// seen null inputs and which groups have seen any inputs, for + /// [`BooleanArray`]s. + /// + /// Since `BooleanArray` is not a [`PrimitiveArray`] it must be + /// handled specially. + /// + /// See [`Self::accumulate`], which handles `PrimitiveArray`s, for + /// more details on other arguments. + pub fn accumulate_boolean( + &mut self, + group_indices: &[usize], + values: &BooleanArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + mut value_fn: F, + ) where + F: FnMut(usize, bool) + Send, + { + let data = values.values(); + assert_eq!(data.len(), group_indices.len()); + + // ensure the seen_values is big enough (start everything at + // "not seen" valid) + let seen_values = + initialize_builder(&mut self.seen_values, total_num_groups, false); + + // These could be made more performant by iterating in chunks of 64 bits at a time + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + // if we have previously seen nulls, ensure the null + // buffer is big enough (start everything at valid) + group_indices.iter().zip(data.iter()).for_each( + |(&group_index, new_value)| { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value) + }, + ) + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + group_indices + .iter() + .zip(data.iter()) + .zip(nulls.iter()) + .for_each(|((&group_index, new_value), is_valid)| { + if is_valid { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value); + } + }) + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, new_value), filter_value)| { + if let Some(true) = filter_value { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + seen_values.set_bit(group_index, true); + value_fn(group_index, new_value) + } + } + }) + } + } + } + + /// Creates the final [`NullBuffer`] representing which + /// group_indices should have null values (because they never saw + /// any values) + /// + /// resets the internal state to empty + pub fn build(&mut self) -> NullBuffer { + NullBuffer::new(self.seen_values.finish()) + } +} + +/// This function is called to update the accumulator state per row +/// when the value is not needed (e.g. COUNT) +/// +/// `F`: Invoked like `value_fn(group_index) for all non null values +/// passing the filter. Note that no tracking is done for null inputs +/// or which groups have seen any values +/// +/// See [`NullState::accumulate`], for more details on other +/// arguments. +pub fn accumulate_indices( + group_indices: &[usize], + nulls: Option<&NullBuffer>, + opt_filter: Option<&BooleanArray>, + mut index_fn: F, +) where + F: FnMut(usize) + Send, +{ + match (nulls, opt_filter) { + (None, None) => { + for &group_index in group_indices.iter() { + index_fn(group_index) + } + } + (None, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + let iter = group_indices.iter().zip(filter.iter()); + for (&group_index, filter_value) in iter { + if let Some(true) = filter_value { + index_fn(group_index) + } + } + } + (Some(valids), None) => { + assert_eq!(valids.len(), group_indices.len()); + // This is based on (ahem, COPY/PASTA) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let bit_chunks = valids.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + + group_indices_chunks.zip(bit_chunks.iter()).for_each( + |(group_index_chunk, mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().for_each(|&group_index| { + // valid bit was set, real vale + let is_valid = (mask & index_mask) != 0; + if is_valid { + index_fn(group_index); + } + index_mask <<= 1; + }) + }, + ); + + // handle any remaining bits (after the intial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .enumerate() + .for_each(|(i, &group_index)| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + index_fn(group_index) + } + }); + } + + (Some(valids), Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + assert_eq!(valids.len(), group_indices.len()); + // The performance with a filter could likely be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(valids.iter()) + .for_each(|((filter_value, &group_index), is_valid)| { + if let (Some(true), true) = (filter_value, is_valid) { + index_fn(group_index) + } + }) + } + } +} + +/// Ensures that `builder` contains a `BooleanBufferBuilder with at +/// least `total_num_groups`. +/// +/// All new entries are initialized to `default_value` +fn initialize_builder( + builder: &mut BooleanBufferBuilder, + total_num_groups: usize, + default_value: bool, +) -> &mut BooleanBufferBuilder { + if builder.len() < total_num_groups { + let new_groups = total_num_groups - builder.len(); + builder.append_n(new_groups, default_value); + } + builder +} + +#[cfg(test)] +mod test { + use super::*; + + use arrow_array::UInt32Array; + use arrow_buffer::BooleanBuffer; + use hashbrown::HashSet; + use rand::{rngs::ThreadRng, Rng}; + + #[test] + fn accumulate() { + let group_indices = (0..100).collect(); + let values = (0..100).map(|i| (i + 1) * 10).collect(); + let values_with_nulls = (0..100) + .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) + .collect(); + + // default to every fifth value being false, every even + // being null + let filter: BooleanArray = (0..100) + .map(|i| { + let is_even = i % 2 == 0; + let is_fifth = i % 5 == 0; + if is_even { + None + } else if is_fifth { + Some(false) + } else { + Some(true) + } + }) + .collect(); + + Fixture { + group_indices, + values, + values_with_nulls, + filter, + } + .run() + } + + #[test] + fn accumulate_fuzz() { + let mut rng = rand::thread_rng(); + for _ in 0..100 { + Fixture::new_random(&mut rng).run(); + } + } + + /// Values for testing (there are enough values to exercise the 64 bit chunks + struct Fixture { + /// 100..0 + group_indices: Vec, + + /// 10, 20, ... 1010 + values: Vec, + + /// same as values, but every third is null: + /// None, Some(20), Some(30), None ... + values_with_nulls: Vec>, + + /// filter (defaults to None) + filter: BooleanArray, + } + + impl Fixture { + fn new_random(rng: &mut ThreadRng) -> Self { + // Number of input values in a batch + let num_values: usize = rng.gen_range(1..200); + // number of distinct groups + let num_groups: usize = rng.gen_range(2..1000); + let max_group = num_groups - 1; + + let group_indices: Vec = (0..num_values) + .map(|_| rng.gen_range(0..max_group)) + .collect(); + + let values: Vec = (0..num_values).map(|_| rng.gen()).collect(); + + // 10% chance of false + // 10% change of null + // 80% chance of true + let filter: BooleanArray = (0..num_values) + .map(|_| { + let filter_value = rng.gen_range(0.0..1.0); + if filter_value < 0.1 { + Some(false) + } else if filter_value < 0.2 { + None + } else { + Some(true) + } + }) + .collect(); + + // random values with random number and location of nulls + // random null percentage + let null_pct: f32 = rng.gen_range(0.0..1.0); + let values_with_nulls: Vec> = (0..num_values) + .map(|_| { + let is_null = null_pct < rng.gen_range(0.0..1.0); + if is_null { + None + } else { + Some(rng.gen()) + } + }) + .collect(); + + Self { + group_indices, + values, + values_with_nulls, + filter, + } + } + + /// returns `Self::values` an Array + fn values_array(&self) -> UInt32Array { + UInt32Array::from(self.values.clone()) + } + + /// returns `Self::values_with_nulls` as an Array + fn values_with_nulls_array(&self) -> UInt32Array { + UInt32Array::from(self.values_with_nulls.clone()) + } + + /// Calls `NullState::accumulate` and `accumulate_indices` + /// with all combinations of nulls and filter values + fn run(&self) { + let total_num_groups = *self.group_indices.iter().max().unwrap() + 1; + + let group_indices = &self.group_indices; + let values_array = self.values_array(); + let values_with_nulls_array = self.values_with_nulls_array(); + let filter = &self.filter; + + // no null, no filters + Self::accumulate_test(group_indices, &values_array, None, total_num_groups); + + // nulls, no filters + Self::accumulate_test( + group_indices, + &values_with_nulls_array, + None, + total_num_groups, + ); + + // no nulls, filters + Self::accumulate_test( + group_indices, + &values_array, + Some(filter), + total_num_groups, + ); + + // nulls, filters + Self::accumulate_test( + group_indices, + &values_with_nulls_array, + Some(filter), + total_num_groups, + ); + } + + /// Calls `NullState::accumulate` and `accumulate_indices` to + /// ensure it generates the correct values. + /// + fn accumulate_test( + group_indices: &[usize], + values: &UInt32Array, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) { + Self::accumulate_values_test( + group_indices, + values, + opt_filter, + total_num_groups, + ); + Self::accumulate_indices_test(group_indices, values.nulls(), opt_filter); + + // Convert values into a boolean array (anything above the + // average is true, otherwise false) + let avg: usize = values.iter().filter_map(|v| v.map(|v| v as usize)).sum(); + let boolean_values: BooleanArray = + values.iter().map(|v| v.map(|v| v as usize > avg)).collect(); + Self::accumulate_boolean_test( + group_indices, + &boolean_values, + opt_filter, + total_num_groups, + ); + } + + /// This is effectively a different implementation of + /// accumulate that we compare with the above implementation + fn accumulate_values_test( + group_indices: &[usize], + values: &UInt32Array, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) { + let mut accumulated_values = vec![]; + let mut null_state = NullState::new(); + + null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_index, value)); + }, + ); + + // Figure out the expected values + let mut expected_values = vec![]; + let mut mock = MockNullState::new(); + + match opt_filter { + None => group_indices.iter().zip(values.iter()).for_each( + |(&group_index, value)| { + if let Some(value) = value { + mock.saw_value(group_index); + expected_values.push((group_index, value)); + } + }, + ), + Some(filter) => { + group_indices + .iter() + .zip(values.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, value), is_included)| { + // if value passed filter + if let Some(true) = is_included { + if let Some(value) = value { + mock.saw_value(group_index); + expected_values.push((group_index, value)); + } + } + }); + } + } + + assert_eq!(accumulated_values, expected_values, + "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); + let seen_values = null_state.seen_values.finish_cloned(); + mock.validate_seen_values(&seen_values); + + // Validate the final buffer (one value per group) + let expected_null_buffer = mock.expected_null_buffer(total_num_groups); + + let null_buffer = null_state.build(); + + assert_eq!(null_buffer, expected_null_buffer); + } + + // Calls `accumulate_indices` + // and opt_filter and ensures it calls the right values + fn accumulate_indices_test( + group_indices: &[usize], + nulls: Option<&NullBuffer>, + opt_filter: Option<&BooleanArray>, + ) { + let mut accumulated_values = vec![]; + + accumulate_indices(group_indices, nulls, opt_filter, |group_index| { + accumulated_values.push(group_index); + }); + + // Figure out the expected values + let mut expected_values = vec![]; + + match (nulls, opt_filter) { + (None, None) => group_indices.iter().for_each(|&group_index| { + expected_values.push(group_index); + }), + (Some(nulls), None) => group_indices.iter().zip(nulls.iter()).for_each( + |(&group_index, is_valid)| { + if is_valid { + expected_values.push(group_index); + } + }, + ), + (None, Some(filter)) => group_indices.iter().zip(filter.iter()).for_each( + |(&group_index, is_included)| { + if let Some(true) = is_included { + expected_values.push(group_index); + } + }, + ), + (Some(nulls), Some(filter)) => { + group_indices + .iter() + .zip(nulls.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, is_valid), is_included)| { + // if value passed filter + if let (true, Some(true)) = (is_valid, is_included) { + expected_values.push(group_index); + } + }); + } + } + + assert_eq!(accumulated_values, expected_values, + "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); + } + + /// This is effectively a different implementation of + /// accumulate_boolean that we compare with the above implementation + fn accumulate_boolean_test( + group_indices: &[usize], + values: &BooleanArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) { + let mut accumulated_values = vec![]; + let mut null_state = NullState::new(); + + null_state.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_index, value)); + }, + ); + + // Figure out the expected values + let mut expected_values = vec![]; + let mut mock = MockNullState::new(); + + match opt_filter { + None => group_indices.iter().zip(values.iter()).for_each( + |(&group_index, value)| { + if let Some(value) = value { + mock.saw_value(group_index); + expected_values.push((group_index, value)); + } + }, + ), + Some(filter) => { + group_indices + .iter() + .zip(values.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, value), is_included)| { + // if value passed filter + if let Some(true) = is_included { + if let Some(value) = value { + mock.saw_value(group_index); + expected_values.push((group_index, value)); + } + } + }); + } + } + + assert_eq!(accumulated_values, expected_values, + "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); + + let seen_values = null_state.seen_values.finish_cloned(); + mock.validate_seen_values(&seen_values); + + // Validate the final buffer (one value per group) + let expected_null_buffer = mock.expected_null_buffer(total_num_groups); + + let null_buffer = null_state.build(); + + assert_eq!(null_buffer, expected_null_buffer); + } + } + + /// Parallel implementaiton of NullState to check expected values + #[derive(Debug, Default)] + struct MockNullState { + /// group indices that had values that passed the filter + seen_values: HashSet, + } + + impl MockNullState { + fn new() -> Self { + Default::default() + } + + fn saw_value(&mut self, group_index: usize) { + self.seen_values.insert(group_index); + } + + /// did this group index see any input? + fn expected_seen(&self, group_index: usize) -> bool { + self.seen_values.contains(&group_index) + } + + /// Validate that the seen_values matches self.seen_values + fn validate_seen_values(&self, seen_values: &BooleanBuffer) { + for (group_index, is_seen) in seen_values.iter().enumerate() { + let expected_seen = self.expected_seen(group_index); + assert_eq!( + expected_seen, is_seen, + "mismatch at for group {group_index}" + ); + } + } + + /// Create the expected null buffer based on if the input had nulls and a filter + fn expected_null_buffer(&self, total_num_groups: usize) -> NullBuffer { + (0..total_num_groups) + .map(|group_index| self.expected_seen(group_index)) + .collect() + } + } +} diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs new file mode 100644 index 000000000000..7b4c61fe7dc4 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs @@ -0,0 +1,355 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Adapter that makes [`GroupsAccumulator`] out of [`Accumulator`] + +use super::GroupsAccumulator; +use arrow::{ + array::{AsArray, UInt32Builder}, + compute, + datatypes::UInt32Type, +}; +use arrow_array::{ArrayRef, BooleanArray, PrimitiveArray}; +use datafusion_common::{ + utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, +}; +use datafusion_expr::Accumulator; + +/// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] +/// +/// While [`Accumulator`] are simpler to implement and can support +/// more general calculations (like retractable window functions), +/// they are not as fast as a specialized `GroupsAccumulator`. This +/// interface bridges the gap so the group by operator only operates +/// in terms of [`Accumulator`]. +pub struct GroupsAccumulatorAdapter { + factory: Box Result> + Send>, + + /// state for each group, stored in group_index order + states: Vec, + + /// Current memory usage, in bytes. + /// + /// Note this is incrementally updated to avoid size() being a + /// bottleneck, which we saw in earlier implementations. + allocation_bytes: usize, +} + +struct AccumulatorState { + /// [`Accumulator`] that stores the per-group state + accumulator: Box, + + // scratch space: indexes in the input array that will be fed to + // this accumulator. Stores indexes as `u32` to match the arrow + // `take` kernel input. + indices: Vec, +} + +impl AccumulatorState { + fn new(accumulator: Box) -> Self { + Self { + accumulator, + indices: vec![], + } + } + + /// Returns the amount of memory taken by this structre and its accumulator + fn size(&self) -> usize { + self.accumulator.size() + + std::mem::size_of_val(self) + + std::mem::size_of::() * self.indices.capacity() + } +} + +impl GroupsAccumulatorAdapter { + /// Create a new adapter that will create a new [`Accumulator`] + /// for each group, using the specified factory function + pub fn new(factory: F) -> Self + where + F: Fn() -> Result> + Send + 'static, + { + let mut new_self = Self { + factory: Box::new(factory), + states: vec![], + allocation_bytes: 0, + }; + new_self.reset_allocation(); + new_self + } + + // Reset the allocation bytes to empty state + fn reset_allocation(&mut self) { + assert!(self.states.is_empty()); + self.allocation_bytes = std::mem::size_of::(); + } + + /// Ensure that self.accumulators has total_num_groups + fn make_accumulators_if_needed(&mut self, total_num_groups: usize) -> Result<()> { + // can't shrink + assert!(total_num_groups >= self.states.len()); + let vec_size_pre = + std::mem::size_of::() * self.states.capacity(); + + // instantiate new accumulators + let new_accumulators = total_num_groups - self.states.len(); + for _ in 0..new_accumulators { + let accumulator = (self.factory)()?; + let state = AccumulatorState::new(accumulator); + self.allocation_bytes += state.size(); + self.states.push(state); + } + + self.allocation_bytes += + std::mem::size_of::() * self.states.capacity(); + self.allocation_bytes -= vec_size_pre; + Ok(()) + } + + /// invokes f(accumulator, values) for each group that has values + /// in group_indices. + /// + /// This function first reorders the input and filter so that + /// values for each group_index are contiguous and then invokes f + /// on the contiguous ranges, to minimize per-row overhead + /// + /// ```text + /// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ + /// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ ┏━━━━━┓ │ ┌─────┐ │ ┌─────┐ + /// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ ┃ 0 ┃ │ │ 200 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ ┃ 0 ┃ │ │ 300 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ ┃ 1 ┃ │ │ 200 │ │ │ │NULL │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ────────▶ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ ┃ 2 ┃ │ │ 200 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ ┃ 2 ┃ │ │ 100 │ │ │ │ f │ │ + /// │ └─────┘ │ │ └─────┘ │ └─────┘ ┗━━━━━┛ │ └─────┘ │ └─────┘ + /// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ └─────────┘ └ ─ ─ ─ ─ ┘ + /// + /// logical group values opt_filter logical group values opt_filter + /// + /// ``` + fn invoke_per_accumulator( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + f: F, + ) -> Result<()> + where + F: Fn(&mut dyn Accumulator, &[ArrayRef]) -> Result<()>, + { + self.make_accumulators_if_needed(total_num_groups)?; + + assert_eq!(values[0].len(), group_indices.len()); + + // figure out which input rows correspond to which groups. + // Note that self.state.indices starts empty for all groups + // (it is cleared out below) + for (idx, group_index) in group_indices.iter().enumerate() { + self.states[*group_index].indices.push(idx as u32); + } + + // groups_with_rows holds a list of group indexes that have + // any rows that need to be accumulated, stored in order of + // group_index + + let mut groups_with_rows = vec![]; + + // batch_indices holds indices into values, each group is contiguous + let mut batch_indices = UInt32Builder::with_capacity(0); + + // offsets[i] is index into batch_indices where the rows for + // group_index i starts + let mut offsets = vec![0]; + + let mut offset_so_far = 0; + for (group_index, state) in self.states.iter_mut().enumerate() { + let indices = &state.indices; + if indices.is_empty() { + continue; + } + + groups_with_rows.push(group_index); + batch_indices.append_slice(indices); + offset_so_far += indices.len(); + offsets.push(offset_so_far); + } + let batch_indices = batch_indices.finish(); + + // reorder the values and opt_filter by batch_indices so that + // all values for each group are contiguous, then invoke the + // accumulator once per group with values + let values = get_arrayref_at_indices(values, &batch_indices)?; + let opt_filter = get_filter_at_indices(opt_filter, &batch_indices)?; + + // invoke each accumulator with the appropriate rows, first + // pulling the input arguments for this group into their own + // RecordBatch(es) + let iter = groups_with_rows.iter().zip(offsets.windows(2)); + + for (&group_idx, offsets) in iter { + let state = &mut self.states[group_idx]; + let size_pre = state.size(); + + let values_to_accumulate = + slice_and_maybe_filter(&values, opt_filter.as_ref(), offsets)?; + (f)(state.accumulator.as_mut(), &values_to_accumulate)?; + + // clear out the state so they are empty for next + // iteration + state.indices.clear(); + + self.allocation_bytes += state.size(); + self.allocation_bytes -= size_pre; + } + Ok(()) + } +} + +impl GroupsAccumulator for GroupsAccumulatorAdapter { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.invoke_per_accumulator( + values, + group_indices, + opt_filter, + total_num_groups, + |accumulator, values_to_accumulate| { + accumulator.update_batch(values_to_accumulate) + }, + )?; + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let states = std::mem::take(&mut self.states); + + let results: Vec = states + .into_iter() + .map(|state| state.accumulator.evaluate()) + .collect::>()?; + + let result = ScalarValue::iter_to_array(results); + self.reset_allocation(); + result + } + + fn state(&mut self) -> Result> { + let states = std::mem::take(&mut self.states); + + // each accumulator produces a potential vector of values + // which we need to form into columns + let mut results: Vec> = vec![]; + + for state in states { + let accumulator_state = state.accumulator.state()?; + results.resize_with(accumulator_state.len(), Vec::new); + for (idx, state_val) in accumulator_state.into_iter().enumerate() { + results[idx].push(state_val); + } + } + + // create an array for each intermediate column + let arrays = results + .into_iter() + .map(ScalarValue::iter_to_array) + .collect::>>()?; + + // double check each array has the same length (aka the + // accumulator was implemented correctly + if let Some(first_col) = arrays.get(0) { + for arr in &arrays { + assert_eq!(arr.len(), first_col.len()) + } + } + + self.reset_allocation(); + Ok(arrays) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.invoke_per_accumulator( + values, + group_indices, + opt_filter, + total_num_groups, + |accumulator, values_to_accumulate| { + accumulator.merge_batch(values_to_accumulate) + }, + )?; + Ok(()) + } + + fn size(&self) -> usize { + self.allocation_bytes + } +} + +fn get_filter_at_indices( + opt_filter: Option<&BooleanArray>, + indices: &PrimitiveArray, +) -> Result> { + opt_filter + .map(|filter| { + compute::take( + &filter, indices, None, // None: no index check + ) + }) + .transpose() + .map_err(DataFusionError::ArrowError) +} + +// Copied from physical-plan +pub(crate) fn slice_and_maybe_filter( + aggr_array: &[ArrayRef], + filter_opt: Option<&ArrayRef>, + offsets: &[usize], +) -> Result> { + let (offset, length) = (offsets[0], offsets[1] - offsets[0]); + let sliced_arrays: Vec = aggr_array + .iter() + .map(|array| array.slice(offset, length)) + .collect(); + + if let Some(f) = filter_opt { + let filter_array = f.slice(offset, length); + let filter_array = filter_array.as_boolean(); + + sliced_arrays + .iter() + .map(|array| { + compute::filter(array, filter_array).map_err(DataFusionError::ArrowError) + }) + .collect() + } else { + Ok(sliced_arrays) + } +} diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/bool_op.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/bool_op.rs new file mode 100644 index 000000000000..83ffc3717b44 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/bool_op.rs @@ -0,0 +1,127 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::array::AsArray; +use arrow_array::{ArrayRef, BooleanArray}; +use arrow_buffer::BooleanBufferBuilder; +use datafusion_common::Result; + +use crate::GroupsAccumulator; + +use super::accumulate::NullState; + +/// An accumulator that implements a single operation over a +/// [`BooleanArray`] where the accumulated state is also boolean (such +/// as [`BitAndAssign`]) +/// +/// F: The function to apply to two elements. The first argument is +/// the existing value and should be updated with the second value +/// (e.g. [`BitAndAssign`] style). +/// +/// [`BitAndAssign`]: std::ops::BitAndAssign +#[derive(Debug)] +pub struct BooleanGroupsAccumulator +where + F: Fn(bool, bool) -> bool + Send + Sync, +{ + /// values per group + values: BooleanBufferBuilder, + + /// Track nulls in the input / filters + null_state: NullState, + + /// Function that computes the output + bool_fn: F, +} + +impl BooleanGroupsAccumulator +where + F: Fn(bool, bool) -> bool + Send + Sync, +{ + pub fn new(bitop_fn: F) -> Self { + Self { + values: BooleanBufferBuilder::new(0), + null_state: NullState::new(), + bool_fn: bitop_fn, + } + } +} + +impl GroupsAccumulator for BooleanGroupsAccumulator +where + F: Fn(bool, bool) -> bool + Send + Sync, +{ + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_boolean(); + + if self.values.len() < total_num_groups { + let new_groups = total_num_groups - self.values.len(); + self.values.append_n(new_groups, Default::default()); + } + + // NullState dispatches / handles tracking nulls and groups that saw no values + self.null_state.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let current_value = self.values.get_bit(group_index); + let value = (self.bool_fn)(current_value, new_value); + self.values.set_bit(group_index, value); + }, + ); + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let values = self.values.finish(); + let nulls = self.null_state.build(); + let values = BooleanArray::new(values, Some(nulls)); + Ok(Arc::new(values)) + } + + fn state(&mut self) -> Result> { + self.evaluate().map(|arr| vec![arr]) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + // update / merge are the same + self.update_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn size(&self) -> usize { + // capacity is in bits, so convert to bytes + self.values.capacity() / 8 + self.null_state.size() + } +} diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs new file mode 100644 index 000000000000..49d62e7a9394 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Vectorized [`GroupsAccumulator`] + +pub(crate) mod accumulate; +mod adapter; +pub use adapter::GroupsAccumulatorAdapter; + +pub(crate) mod bool_op; +pub(crate) mod prim_op; + +use arrow_array::{ArrayRef, BooleanArray}; +use datafusion_common::Result; + +/// `GroupAccumulator` implements a single aggregate (e.g. AVG) and +/// stores the state for *all* groups internally. +/// +/// Each group is assigned a `group_index` by the hash table and each +/// accumulator manages the specific state, one per group_index. +/// +/// group_indexes are contiguous (there aren't gaps), and thus it is +/// expected that each GroupAccumulator will use something like `Vec<..>` +/// to store the group states. +pub trait GroupsAccumulator: Send { + /// Updates the accumulator's state from its arguments, encoded as + /// a vector of [`ArrayRef`]s. + /// + /// * `values`: the input arguments to the accumulator + /// + /// * `group_indices`: To which groups do the rows in `values` + /// belong, group id) + /// + /// * `opt_filter`: if present, only update aggregate state using + /// `values[i]` if `opt_filter[i]` is true + /// + /// * `total_num_groups`: the number of groups (the largest + /// group_index is thus `total_num_groups - 1`). + /// + /// Note that subsequent calls to update_batch may have larger + /// total_num_groups as new groups are seen. + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()>; + + /// Returns the final aggregate value for each group as a single + /// `RecordBatch`, resetting the internal state. + /// + /// The rows returned *must* be in group_index order: The value + /// for group_index 0, followed by 1, etc. Any group_index that + /// did not have values, should be null. + /// + /// For example, a `SUM` accumulator maintains a running sum for + /// each group, and `evaluate` will produce that running sum as + /// its output for all groups, in group_index order + /// + /// The accumulator should free to release / reset it is internal + /// state after this call to the same as it was after being + /// initially created. + fn evaluate(&mut self) -> Result; + + /// Returns the intermediate aggregate state for this accumulator, + /// used for multi-phase grouping, resetting its internal state. + /// + /// The rows returned *must* be in group_index order: The value + /// for group_index 0, followed by 1, etc. Any group_index that + /// did not have values, should be null. + /// + /// For example, `AVG` might return two arrays: `SUM` and `COUNT` + /// but the `MIN` aggregate would just return a single array. + /// + /// Note more sophisticated internal state can be passed as + /// single `StructArray` rather than multiple arrays. + /// + /// The accumulator should free to release / reset its internal + /// state after this call to the same as it was after being + /// initially created. + fn state(&mut self) -> Result>; + + /// Merges intermediate state (the output from [`Self::state`]) + /// into this accumulator's values. + /// + /// For some aggregates (such as `SUM`), `merge_batch` is the same + /// as `update_batch`, but for some aggregates (such as `COUNT`, + /// where the partial counts must be summed) the operations + /// differ. See [`Self::state`] for more details on how state is + /// used and merged. + /// + /// * `values`: arrays produced from calling `state` previously to the accumulator + /// + /// Other arguments are the same as for [`Self::update_batch`]; + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()>; + + /// Amount of memory used to store the state of this accumulator, + /// in bytes. This function is called once per batch, so it should + /// be `O(n)` to compute, not `O(num_groups)` + fn size(&self) -> usize; +} diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/prim_op.rs new file mode 100644 index 000000000000..860301078909 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/prim_op.rs @@ -0,0 +1,131 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::{array::AsArray, datatypes::ArrowPrimitiveType}; +use arrow_array::{ArrayRef, BooleanArray, PrimitiveArray}; +use arrow_schema::DataType; +use datafusion_common::Result; + +use crate::{aggregate::utils::adjust_output_array, GroupsAccumulator}; + +use super::accumulate::NullState; + +/// An accumulator that implements a single operation over +/// [`ArrowPrimitiveType`] where the accumulated state is the same as +/// the input type (such as `Sum`) +/// +/// F: The function to apply to two elements. The first argument is +/// the existing value and should be updated with the second value +/// (e.g. [`BitAndAssign`] style). +/// +/// [`BitAndAssign`]: std::ops::BitAndAssign +#[derive(Debug)] +pub struct PrimitiveGroupsAccumulator +where + T: ArrowPrimitiveType + Send, + F: Fn(&mut T::Native, T::Native) + Send + Sync, +{ + /// values per group, stored as the native type + values: Vec, + + /// The output type (needed for Decimal precision and scale) + data_type: DataType, + + /// Track nulls in the input / filters + null_state: NullState, + + /// Function that computes the primitive result + prim_fn: F, +} + +impl PrimitiveGroupsAccumulator +where + T: ArrowPrimitiveType + Send, + F: Fn(&mut T::Native, T::Native) + Send + Sync, +{ + pub fn new(data_type: &DataType, prim_fn: F) -> Self { + Self { + values: vec![], + data_type: data_type.clone(), + null_state: NullState::new(), + prim_fn, + } + } +} + +impl GroupsAccumulator for PrimitiveGroupsAccumulator +where + T: ArrowPrimitiveType + Send, + F: Fn(&mut T::Native, T::Native) + Send + Sync, +{ + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + + // update values + self.values.resize(total_num_groups, T::default_value()); + + // NullState dispatches / handles tracking nulls and groups that saw no values + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let value = &mut self.values[group_index]; + (self.prim_fn)(value, new_value); + }, + ); + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let values = std::mem::take(&mut self.values); + let nulls = self.null_state.build(); + let values = PrimitiveArray::::new(values.into(), Some(nulls)); // no copy + + adjust_output_array(&self.data_type, Arc::new(values)) + } + + fn state(&mut self) -> Result> { + self.evaluate().map(|arr| vec![arr]) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + // update / merge are the same + self.update_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn size(&self) -> usize { + self.values.capacity() * std::mem::size_of::() + self.null_state.size() + } +} diff --git a/datafusion/physical-expr/src/aggregate/min_max.rs b/datafusion/physical-expr/src/aggregate/min_max.rs index e3c061dc1354..ebf317e6d0f3 100644 --- a/datafusion/physical-expr/src/aggregate/min_max.rs +++ b/datafusion/physical-expr/src/aggregate/min_max.rs @@ -21,9 +21,13 @@ use std::any::Any; use std::convert::TryFrom; use std::sync::Arc; -use crate::{AggregateExpr, PhysicalExpr}; +use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr}; use arrow::compute; -use arrow::datatypes::{DataType, TimeUnit}; +use arrow::datatypes::{ + DataType, Date32Type, Date64Type, Time32MillisecondType, Time32SecondType, + Time64MicrosecondType, Time64NanosecondType, TimeUnit, TimestampMicrosecondType, + TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, +}; use arrow::{ array::{ ArrayRef, BooleanArray, Date32Array, Date64Array, Float32Array, Float64Array, @@ -35,9 +39,16 @@ use arrow::{ }, datatypes::Field, }; +use arrow_array::cast::AsArray; +use arrow_array::types::{ + Decimal128Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, + UInt16Type, UInt32Type, UInt64Type, UInt8Type, +}; +use arrow_array::{ArrowNumericType, PrimitiveArray}; use datafusion_common::ScalarValue; use datafusion_common::{downcast_value, DataFusionError, Result}; use datafusion_expr::Accumulator; +use log::debug; use crate::aggregate::row_accumulator::{ is_row_accumulator_support_dtype, RowAccumulator, @@ -48,7 +59,9 @@ use arrow::array::Array; use arrow::array::Decimal128Array; use datafusion_row::accessor::RowAccessor; +use super::groups_accumulator::accumulate::NullState; use super::moving_min_max; +use super::utils::adjust_output_array; // Min/max aggregation can take Dictionary encode input but always produces unpacked // (aka non Dictionary) output. We need to adjust the output data type to reflect this. @@ -87,6 +100,15 @@ impl Max { } } +macro_rules! instantiate_min_max_accumulator { + ($SELF:expr, $NUMERICTYPE:ident, $MIN:expr) => {{ + Ok(Box::new(MinMaxGroupsPrimitiveAccumulator::< + $NUMERICTYPE, + $MIN, + >::new(&$SELF.data_type))) + }}; +} + impl AggregateExpr for Max { /// Return a reference to Any that can be used for downcasting fn as_any(&self) -> &dyn Any { @@ -125,6 +147,28 @@ impl AggregateExpr for Max { is_row_accumulator_support_dtype(&self.data_type) } + fn groups_accumulator_supported(&self) -> bool { + use DataType::*; + matches!( + self.data_type, + Int8 | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float32 + | Float64 + | Decimal128(_, _) + | Date32 + | Date64 + | Time32(_) + | Time64(_) + | Timestamp(_, _) + ) + } + fn create_row_accumulator( &self, start_index: usize, @@ -135,6 +179,66 @@ impl AggregateExpr for Max { ))) } + fn create_groups_accumulator(&self) -> Result> { + use DataType::*; + use TimeUnit::*; + + match self.data_type { + Int8 => instantiate_min_max_accumulator!(self, Int8Type, false), + Int16 => instantiate_min_max_accumulator!(self, Int16Type, false), + Int32 => instantiate_min_max_accumulator!(self, Int32Type, false), + Int64 => instantiate_min_max_accumulator!(self, Int64Type, false), + UInt8 => instantiate_min_max_accumulator!(self, UInt8Type, false), + UInt16 => instantiate_min_max_accumulator!(self, UInt16Type, false), + UInt32 => instantiate_min_max_accumulator!(self, UInt32Type, false), + UInt64 => instantiate_min_max_accumulator!(self, UInt64Type, false), + Float32 => { + instantiate_min_max_accumulator!(self, Float32Type, false) + } + Float64 => { + instantiate_min_max_accumulator!(self, Float64Type, false) + } + Date32 => instantiate_min_max_accumulator!(self, Date32Type, false), + Date64 => instantiate_min_max_accumulator!(self, Date64Type, false), + Time32(Second) => { + instantiate_min_max_accumulator!(self, Time32SecondType, false) + } + Time32(Millisecond) => { + instantiate_min_max_accumulator!(self, Time32MillisecondType, false) + } + Time64(Microsecond) => { + instantiate_min_max_accumulator!(self, Time64MicrosecondType, false) + } + Time64(Nanosecond) => { + instantiate_min_max_accumulator!(self, Time64NanosecondType, false) + } + Timestamp(Second, _) => { + instantiate_min_max_accumulator!(self, TimestampSecondType, false) + } + Timestamp(Millisecond, _) => { + instantiate_min_max_accumulator!(self, TimestampMillisecondType, false) + } + Timestamp(Microsecond, _) => { + instantiate_min_max_accumulator!(self, TimestampMicrosecondType, false) + } + Timestamp(Nanosecond, _) => { + instantiate_min_max_accumulator!(self, TimestampNanosecondType, false) + } + + // It would be nice to have a fast implementation for Strings as well + // https://github.com/apache/arrow-datafusion/issues/6906 + Decimal128(_, _) => Ok(Box::new(MinMaxGroupsPrimitiveAccumulator::< + Decimal128Type, + false, + >::new(&self.data_type))), + // This is only reached if groups_accumulator_supported is out of sync + _ => Err(DataFusionError::Internal(format!( + "MinMaxGroupsPrimitiveAccumulator not supported for max({})", + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } @@ -835,6 +939,84 @@ impl AggregateExpr for Min { ))) } + fn groups_accumulator_supported(&self) -> bool { + use DataType::*; + matches!( + self.data_type, + Int8 | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float32 + | Float64 + | Decimal128(_, _) + | Date32 + | Date64 + | Time32(_) + | Time64(_) + | Timestamp(_, _) + ) + } + + fn create_groups_accumulator(&self) -> Result> { + use DataType::*; + use TimeUnit::*; + match self.data_type { + Int8 => instantiate_min_max_accumulator!(self, Int8Type, true), + Int16 => instantiate_min_max_accumulator!(self, Int16Type, true), + Int32 => instantiate_min_max_accumulator!(self, Int32Type, true), + Int64 => instantiate_min_max_accumulator!(self, Int64Type, true), + UInt8 => instantiate_min_max_accumulator!(self, UInt8Type, true), + UInt16 => instantiate_min_max_accumulator!(self, UInt16Type, true), + UInt32 => instantiate_min_max_accumulator!(self, UInt32Type, true), + UInt64 => instantiate_min_max_accumulator!(self, UInt64Type, true), + Float32 => { + instantiate_min_max_accumulator!(self, Float32Type, true) + } + Float64 => { + instantiate_min_max_accumulator!(self, Float64Type, true) + } + Date32 => instantiate_min_max_accumulator!(self, Date32Type, true), + Date64 => instantiate_min_max_accumulator!(self, Date64Type, true), + Time32(Second) => { + instantiate_min_max_accumulator!(self, Time32SecondType, true) + } + Time32(Millisecond) => { + instantiate_min_max_accumulator!(self, Time32MillisecondType, true) + } + Time64(Microsecond) => { + instantiate_min_max_accumulator!(self, Time64MicrosecondType, true) + } + Time64(Nanosecond) => { + instantiate_min_max_accumulator!(self, Time64NanosecondType, true) + } + Timestamp(Second, _) => { + instantiate_min_max_accumulator!(self, TimestampSecondType, true) + } + Timestamp(Millisecond, _) => { + instantiate_min_max_accumulator!(self, TimestampMillisecondType, true) + } + Timestamp(Microsecond, _) => { + instantiate_min_max_accumulator!(self, TimestampMicrosecondType, true) + } + Timestamp(Nanosecond, _) => { + instantiate_min_max_accumulator!(self, TimestampNanosecondType, true) + } + Decimal128(_, _) => Ok(Box::new(MinMaxGroupsPrimitiveAccumulator::< + Decimal128Type, + true, + >::new(&self.data_type))), + // This is only reached if groups_accumulator_supported is out of sync + _ => Err(DataFusionError::Internal(format!( + "MinMaxGroupsPrimitiveAccumulator not supported for min({})", + self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } @@ -1022,6 +1204,232 @@ impl RowAccumulator for MinRowAccumulator { } } +trait MinMax { + fn min() -> Self; + fn max() -> Self; +} + +impl MinMax for u8 { + fn min() -> Self { + u8::MIN + } + fn max() -> Self { + u8::MAX + } +} +impl MinMax for i8 { + fn min() -> Self { + i8::MIN + } + fn max() -> Self { + i8::MAX + } +} +impl MinMax for u16 { + fn min() -> Self { + u16::MIN + } + fn max() -> Self { + u16::MAX + } +} +impl MinMax for i16 { + fn min() -> Self { + i16::MIN + } + fn max() -> Self { + i16::MAX + } +} +impl MinMax for u32 { + fn min() -> Self { + u32::MIN + } + fn max() -> Self { + u32::MAX + } +} +impl MinMax for i32 { + fn min() -> Self { + i32::MIN + } + fn max() -> Self { + i32::MAX + } +} +impl MinMax for i64 { + fn min() -> Self { + i64::MIN + } + fn max() -> Self { + i64::MAX + } +} +impl MinMax for u64 { + fn min() -> Self { + u64::MIN + } + fn max() -> Self { + u64::MAX + } +} +impl MinMax for f32 { + fn min() -> Self { + f32::MIN + } + fn max() -> Self { + f32::MAX + } +} +impl MinMax for f64 { + fn min() -> Self { + f64::MIN + } + fn max() -> Self { + f64::MAX + } +} +impl MinMax for i128 { + fn min() -> Self { + i128::MIN + } + fn max() -> Self { + i128::MAX + } +} + +/// An accumulator to compute the min or max of a [`PrimitiveArray`]. +/// +/// Stores values as native/primitive type +/// +/// Note this doesn't use [`PrimitiveGroupsAccumulator`] because it +/// needs to control the default accumulator value (which is not +/// `default::Default()`) +/// +/// [`PrimitiveGroupsAccumulator`]: crate::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator +#[derive(Debug)] +struct MinMaxGroupsPrimitiveAccumulator +where + T: ArrowNumericType + Send, + T::Native: MinMax, +{ + /// Min/max per group, stored as the native type + min_max: Vec, + + /// Track nulls in the input / filters + null_state: NullState, + + /// The output datatype (needed for decimal precision/scale) + data_type: DataType, +} + +impl MinMaxGroupsPrimitiveAccumulator +where + T: ArrowNumericType + Send, + T::Native: MinMax, +{ + pub fn new(data_type: &DataType) -> Self { + debug!( + "MinMaxGroupsPrimitiveAccumulator ({}, {})", + std::any::type_name::(), + MIN, + ); + + Self { + min_max: vec![], + null_state: NullState::new(), + data_type: data_type.clone(), + } + } +} + +impl GroupsAccumulator for MinMaxGroupsPrimitiveAccumulator +where + T: ArrowNumericType + Send, + T::Native: MinMax, +{ + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + + self.min_max.resize( + total_num_groups, + if MIN { + T::Native::max() + } else { + T::Native::min() + }, + ); + + // NullState dispatches / handles tracking nulls and groups that saw no values + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let val = &mut self.min_max[group_index]; + match MIN { + true => { + if new_value < *val { + *val = new_value; + } + } + false => { + if new_value > *val { + *val = new_value; + } + } + } + }, + ); + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + Self::update_batch(self, values, group_indices, opt_filter, total_num_groups) + } + + fn evaluate(&mut self) -> Result { + let min_max = std::mem::take(&mut self.min_max); + let nulls = self.null_state.build(); + + let min_max = PrimitiveArray::::new(min_max.into(), Some(nulls)); // no copy + let min_max = adjust_output_array(&self.data_type, Arc::new(min_max))?; + + Ok(Arc::new(min_max)) + } + + // return arrays for min/max values + fn state(&mut self) -> Result> { + let nulls = self.null_state.build(); + + let min_max = std::mem::take(&mut self.min_max); + let min_max = PrimitiveArray::::new(min_max.into(), Some(nulls)); // zero copy + + let min_max = adjust_output_array(&self.data_type, Arc::new(min_max))?; + + Ok(vec![min_max]) + } + + fn size(&self) -> usize { + self.min_max.capacity() * std::mem::size_of::() + self.null_state.size() + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 9be6d5e1ba12..21efb3c2f91b 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -25,6 +25,8 @@ use std::any::Any; use std::fmt::Debug; use std::sync::Arc; +use self::groups_accumulator::GroupsAccumulator; + pub(crate) mod approx_distinct; pub(crate) mod approx_median; pub(crate) mod approx_percentile_cont; @@ -45,6 +47,7 @@ pub(crate) mod median; #[macro_use] pub(crate) mod min_max; pub mod build_in; +pub(crate) mod groups_accumulator; mod hyperloglog; pub mod moving_min_max; pub mod row_accumulator; @@ -118,6 +121,24 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { ))) } + /// If the aggregate expression has a specialized + /// [`GroupsAccumulator`] implementation. If this returns true, + /// `[Self::create_groups_accumulator`] will be called. + fn groups_accumulator_supported(&self) -> bool { + false + } + + /// Return a specialized [`GroupsAccumulator`] that manages state + /// for all groups. + /// + /// For maximum performance, a [`GroupsAccumulator`] should be + /// implemented in addition to [`Accumulator`]. + fn create_groups_accumulator(&self) -> Result> { + Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator hasn't been implemented for {self:?} yet" + ))) + } + /// Construct an expression that calculates the aggregate in reverse. /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). /// For aggregates that do not support calculation in reverse, diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index 29996eaf5cdf..5f00e594fef5 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -19,10 +19,20 @@ use std::any::Any; use std::convert::TryFrom; +use std::ops::AddAssign; use std::sync::Arc; -use crate::{AggregateExpr, PhysicalExpr}; +use super::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use crate::aggregate::row_accumulator::{ + is_row_accumulator_support_dtype, RowAccumulator, +}; +use crate::aggregate::utils::down_cast_any_ref; +use crate::expressions::format_state_name; +use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr}; +use arrow::array::Array; +use arrow::array::Decimal128Array; use arrow::compute; +use arrow::compute::kernels::cast; use arrow::datatypes::DataType; use arrow::{ array::{ @@ -31,17 +41,12 @@ use arrow::{ }, datatypes::Field, }; +use arrow_array::types::{ + Decimal128Type, Float32Type, Float64Type, Int32Type, Int64Type, UInt32Type, + UInt64Type, +}; use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; - -use crate::aggregate::row_accumulator::{ - is_row_accumulator_support_dtype, RowAccumulator, -}; -use crate::aggregate::utils::down_cast_any_ref; -use crate::expressions::format_state_name; -use arrow::array::Array; -use arrow::array::Decimal128Array; -use arrow::compute::cast; use datafusion_row::accessor::RowAccessor; /// SUM aggregate expression @@ -86,6 +91,19 @@ impl Sum { } } +/// Creates a [`PrimitiveGroupsAccumulator`] with the specified +/// [`ArrowPrimitiveType`] which applies `$FN` to each element +/// +/// [`ArrowPrimitiveType`]: arrow::datatypes::ArrowPrimitiveType +macro_rules! instantiate_primitive_accumulator { + ($SELF:expr, $PRIMTYPE:ident, $FN:expr) => {{ + Ok(Box::new(PrimitiveGroupsAccumulator::<$PRIMTYPE, _>::new( + &$SELF.data_type, + $FN, + ))) + }}; +} + impl AggregateExpr for Sum { /// Return a reference to Any that can be used for downcasting fn as_any(&self) -> &dyn Any { @@ -124,6 +142,10 @@ impl AggregateExpr for Sum { is_row_accumulator_support_dtype(&self.data_type) } + fn groups_accumulator_supported(&self) -> bool { + true + } + fn create_row_accumulator( &self, start_index: usize, @@ -134,6 +156,44 @@ impl AggregateExpr for Sum { ))) } + fn create_groups_accumulator(&self) -> Result> { + // instantiate specialized accumulator + match self.data_type { + DataType::UInt64 => { + instantiate_primitive_accumulator!(self, UInt64Type, |x, y| x + .add_assign(y)) + } + DataType::Int64 => { + instantiate_primitive_accumulator!(self, Int64Type, |x, y| x + .add_assign(y)) + } + DataType::UInt32 => { + instantiate_primitive_accumulator!(self, UInt32Type, |x, y| x + .add_assign(y)) + } + DataType::Int32 => { + instantiate_primitive_accumulator!(self, Int32Type, |x, y| x + .add_assign(y)) + } + DataType::Float32 => { + instantiate_primitive_accumulator!(self, Float32Type, |x, y| x + .add_assign(y)) + } + DataType::Float64 => { + instantiate_primitive_accumulator!(self, Float64Type, |x, y| x + .add_assign(y)) + } + DataType::Decimal128(_, _) => { + instantiate_primitive_accumulator!(self, Decimal128Type, |x, y| x + .add_assign(y)) + } + _ => Err(DataFusionError::NotImplemented(format!( + "GroupsAccumulator not supported for {}: {}", + self.name, self.data_type + ))), + } + } + fn reverse_expr(&self) -> Option> { Some(Arc::new(self.clone())) } diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index dbbe0c3f92c0..aada51c9efcd 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -20,6 +20,8 @@ use crate::{AggregateExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; use arrow::datatypes::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; +use arrow_array::cast::AsArray; +use arrow_array::types::Decimal128Type; use arrow_schema::{DataType, Field}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; @@ -145,6 +147,28 @@ pub fn calculate_result_decimal_for_avg( } } +/// Adjust array type metadata if needed +/// +/// Since `Decimal128Arrays` created from `Vec` have +/// default precision and scale, this function adjusts the output to +/// match `data_type`, if necessary +pub fn adjust_output_array( + data_type: &DataType, + array: ArrayRef, +) -> Result { + let array = match data_type { + DataType::Decimal128(p, s) => Arc::new( + array + .as_primitive::() + .clone() + .with_precision_and_scale(*p, *s)?, + ), + // no adjustment needed for other arrays + _ => array, + }; + Ok(array) +} + /// Downcast a `Box` or `Arc` /// and return the inner trait object as [`Any`](std::any::Any) so /// that it can be downcast to a specific implementation. diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 1484cf7ff52c..b695ee169eed 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -47,7 +47,9 @@ pub mod var_provider; pub mod window; // reexport this to maintain compatibility with anything that used from_slice previously +pub use aggregate::groups_accumulator::{GroupsAccumulator, GroupsAccumulatorAdapter}; pub use aggregate::AggregateExpr; + pub use equivalence::{ project_equivalence_properties, project_ordering_equivalence_properties, EquivalenceProperties, EquivalentClass, OrderingEquivalenceProperties, diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 32001b966427..4229e3af708c 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -63,7 +63,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | | datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | | datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | | datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail |