From 8f4427aadf060c328efb5302aee81143b6e20f4c Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 9 Jul 2024 01:55:55 +0530 Subject: [PATCH 01/14] add cargo machete to remove udeps (#641) --- .github/actions/rust-test/action.yaml | 6 +++ native/Cargo.lock | 74 ++------------------------- native/core/Cargo.toml | 7 --- 3 files changed, 10 insertions(+), 77 deletions(-) diff --git a/.github/actions/rust-test/action.yaml b/.github/actions/rust-test/action.yaml index b543bcf00..3aeba0666 100644 --- a/.github/actions/rust-test/action.yaml +++ b/.github/actions/rust-test/action.yaml @@ -37,6 +37,12 @@ runs: cd native cargo check --benches + - name: Check unused dependencies + shell: bash + run: | + cd native + cargo install cargo-machete && cargo machete + - name: Cache Maven dependencies uses: actions/cache@v4 with: diff --git a/native/Cargo.lock b/native/Cargo.lock index ccb6433d9..6136e0339 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -843,11 +843,9 @@ dependencies = [ "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-string", "assertables", "async-trait", "brotli", - "byteorder", "bytes", "chrono", "chrono-tz 0.8.6", @@ -862,7 +860,6 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown", "hex", "itertools 0.11.0", "jni", @@ -873,9 +870,7 @@ dependencies = [ "mimalloc", "num", "once_cell", - "parking_lot", "parquet", - "parquet-format", "paste", "pprof", "prost 0.12.6", @@ -887,9 +882,7 @@ dependencies = [ "snap", "tempfile", "thiserror", - "thrift 0.17.0", "tokio", - "tokio-stream", "twox-hash", "unicode-segmentation", "zstd", @@ -1500,12 +1493,6 @@ dependencies = [ "web-sys", ] -[[package]] -name = "integer-encoding" -version = "1.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" - [[package]] name = "integer-encoding" version = "3.0.4" @@ -1994,15 +1981,6 @@ version = "11.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" -[[package]] -name = "ordered-float" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] - [[package]] name = "ordered-float" version = "2.10.1" @@ -2049,19 +2027,10 @@ dependencies = [ "num-bigint", "paste", "seq-macro", - "thrift 0.17.0", + "thrift", "twox-hash", ] -[[package]] -name = "parquet-format" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" -dependencies = [ - "thrift 0.13.0", -] - [[package]] name = "parse-zoneinfo" version = "0.3.1" @@ -2486,7 +2455,7 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float 2.10.1", + "ordered-float", "serde", ] @@ -2745,28 +2714,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "threadpool" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" -dependencies = [ - "num_cpus", -] - -[[package]] -name = "thrift" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" -dependencies = [ - "byteorder", - "integer-encoding 1.1.7", - "log", - "ordered-float 1.1.1", - "threadpool", -] - [[package]] name = "thrift" version = "0.17.0" @@ -2774,10 +2721,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", - "integer-encoding 3.0.4", - "log", - "ordered-float 2.10.1", - "threadpool", + "integer-encoding", + "ordered-float", ] [[package]] @@ -2838,17 +2783,6 @@ dependencies = [ "syn 2.0.68", ] -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - [[package]] name = "tracing" version = "0.1.40" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index c3e924a44..bd0a3d5e4 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -32,19 +32,16 @@ include = [ ] [dependencies] -parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] } arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } -tokio-stream = { version = "0.1.8" } async-trait = "0.1" log = "0.4" log4rs = "1.2.0" @@ -52,9 +49,7 @@ thiserror = "1" serde = { version = "1", features = ["derive"] } lazy_static = "1.4.0" prost = "0.12.1" -thrift = "0.17" jni = "0.21" -byteorder = "1.4.3" snap = "1.1" brotli = "3.3" flate2 = "1.0" @@ -63,8 +58,6 @@ zstd = "0.11" rand = "0.8" num = "0.4" bytes = "1.5.0" -hashbrown = { version = "0.14", features = ["raw"] } -parking_lot = "0.12" tempfile = "3.8.0" ahash = { version = "0.8", default-features = false } itertools = "0.11.0" From b924aeb5a323c9e9a5442236773086c76e132a73 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 16:52:44 -0700 Subject: [PATCH 02/14] fix: Avoid creating huge duplicate of canonicalized plans for CometNativeExec (#639) * fix: Remove original plan parameter from CometNativeExec * Revert "fix: Remove original plan parameter from CometNativeExec" This reverts commit b272551d76d4bba6b30501b5660420368a874145. * More * Revert "Revert "fix: Remove original plan parameter from CometNativeExec"" This reverts commit 722dc07c75407edbf671d47f23769d4e543dbab5. * More * More * Fix * Fix diffs * Update --- dev/diffs/3.4.3.diff | 4 +- dev/diffs/3.5.1.diff | 4 +- dev/diffs/4.0.0-preview1.diff | 4 +- .../comet/CometSparkSessionExtensions.scala | 68 ++++++++-- .../comet/CometBroadcastExchangeExec.scala | 8 +- .../spark/sql/comet/CometCoalesceExec.scala | 2 + .../sql/comet/CometCollectLimitExec.scala | 4 + .../CometTakeOrderedAndProjectExec.scala | 3 +- .../spark/sql/comet/CometWindowExec.scala | 3 +- .../apache/spark/sql/comet/operators.scala | 46 +++++-- .../q1/simplified.txt | 22 ++-- .../q10/simplified.txt | 22 ++-- .../q11/simplified.txt | 60 ++++----- .../q12/simplified.txt | 18 +-- .../q13/simplified.txt | 30 ++--- .../q14a/simplified.txt | 56 ++++----- .../q14b/simplified.txt | 58 ++++----- .../q15/simplified.txt | 22 ++-- .../q16/simplified.txt | 8 +- .../q17/simplified.txt | 46 +++---- .../q18/simplified.txt | 40 +++--- .../q19/simplified.txt | 30 ++--- .../q2/simplified.txt | 14 +-- .../q20/simplified.txt | 18 +-- .../q21/simplified.txt | 24 ++-- .../q22/simplified.txt | 22 ++-- .../q23a/simplified.txt | 42 +++---- .../q23b/simplified.txt | 48 +++---- .../q24a/simplified.txt | 14 +-- .../q24b/simplified.txt | 14 +-- .../q25/simplified.txt | 46 +++---- .../q26/simplified.txt | 30 ++--- .../q27/simplified.txt | 30 ++--- .../q28/simplified.txt | 24 ++-- .../q29/simplified.txt | 52 ++++---- .../q3/simplified.txt | 16 +-- .../q30/simplified.txt | 28 ++--- .../q31/simplified.txt | 68 +++++----- .../q32/simplified.txt | 20 +-- .../q33/simplified.txt | 50 ++++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q36/simplified.txt | 24 ++-- .../q37/simplified.txt | 22 ++-- .../q38/simplified.txt | 28 ++--- .../q39a/simplified.txt | 38 +++--- .../q39b/simplified.txt | 38 +++--- .../q4/simplified.txt | 84 ++++++------- .../q40/simplified.txt | 10 +- .../q41/simplified.txt | 6 +- .../q42/simplified.txt | 16 +-- .../q43/simplified.txt | 16 +-- .../q44/simplified.txt | 10 +- .../q45/simplified.txt | 28 ++--- .../q46/simplified.txt | 30 ++--- .../q47/simplified.txt | 22 ++-- .../q48/simplified.txt | 26 ++-- .../q49/simplified.txt | 42 +++---- .../q5/simplified.txt | 58 ++++----- .../q50/simplified.txt | 28 ++--- .../q51/simplified.txt | 18 +-- .../q52/simplified.txt | 16 +-- .../q53/simplified.txt | 22 ++-- .../q54/simplified.txt | 32 ++--- .../q55/simplified.txt | 16 +-- .../q56/simplified.txt | 50 ++++---- .../q57/simplified.txt | 22 ++-- .../q58/simplified.txt | 42 +++---- .../q59/simplified.txt | 12 +- .../q6/simplified.txt | 30 ++--- .../q60/simplified.txt | 50 ++++---- .../q61/simplified.txt | 54 ++++---- .../q62/simplified.txt | 28 ++--- .../q63/simplified.txt | 22 ++-- .../q64/simplified.txt | 32 ++--- .../q65/simplified.txt | 22 ++-- .../q66/simplified.txt | 42 +++---- .../q67/simplified.txt | 24 ++-- .../q68/simplified.txt | 30 ++--- .../q69/simplified.txt | 22 ++-- .../q7/simplified.txt | 30 ++--- .../q70/simplified.txt | 24 ++-- .../q71/simplified.txt | 34 ++--- .../q72/simplified.txt | 50 ++++---- .../q73/simplified.txt | 26 ++-- .../q74/simplified.txt | 60 ++++----- .../q75/simplified.txt | 58 ++++----- .../q76/simplified.txt | 30 ++--- .../q77/simplified.txt | 50 ++++---- .../q78/simplified.txt | 14 +-- .../q79/simplified.txt | 26 ++-- .../q8/simplified.txt | 26 ++-- .../q80/simplified.txt | 24 ++-- .../q81/simplified.txt | 28 ++--- .../q82/simplified.txt | 22 ++-- .../q83/simplified.txt | 44 +++---- .../q84/simplified.txt | 28 ++--- .../q85/simplified.txt | 42 +++---- .../q86/simplified.txt | 18 +-- .../q87/simplified.txt | 28 ++--- .../q88/simplified.txt | 104 +++++++-------- .../q89/simplified.txt | 24 ++-- .../q9/simplified.txt | 20 +-- .../q90/simplified.txt | 32 ++--- .../q91/simplified.txt | 42 +++---- .../q92/simplified.txt | 20 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 8 +- .../q95/simplified.txt | 12 +- .../q96/simplified.txt | 20 +-- .../q97/simplified.txt | 10 +- .../q98/simplified.txt | 18 +-- .../q99/simplified.txt | 28 ++--- .../q1/simplified.txt | 18 +-- .../q10/simplified.txt | 22 ++-- .../q11/simplified.txt | 52 ++++---- .../q12/simplified.txt | 16 +-- .../q13/simplified.txt | 28 ++--- .../q14a/simplified.txt | 54 ++++---- .../q14b/simplified.txt | 56 ++++----- .../q15/simplified.txt | 20 +-- .../q16/simplified.txt | 8 +- .../q17/simplified.txt | 44 +++---- .../q18/simplified.txt | 38 +++--- .../q19/simplified.txt | 28 ++--- .../q2/simplified.txt | 12 +- .../q20/simplified.txt | 16 +-- .../q21/simplified.txt | 22 ++-- .../q22/simplified.txt | 20 +-- .../q23a/simplified.txt | 38 +++--- .../q23b/simplified.txt | 44 +++---- .../q24a/simplified.txt | 14 +-- .../q24b/simplified.txt | 14 +-- .../q25/simplified.txt | 44 +++---- .../q26/simplified.txt | 28 ++--- .../q27/simplified.txt | 28 ++--- .../q28/simplified.txt | 12 +- .../q29/simplified.txt | 50 ++++---- .../q3/simplified.txt | 14 +-- .../q30/simplified.txt | 24 ++-- .../q31/simplified.txt | 56 ++++----- .../q32/simplified.txt | 18 +-- .../q33/simplified.txt | 44 +++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q36/simplified.txt | 22 ++-- .../q37/simplified.txt | 22 ++-- .../q38/simplified.txt | 28 ++--- .../q39a/simplified.txt | 34 ++--- .../q39b/simplified.txt | 34 ++--- .../q4/simplified.txt | 72 +++++------ .../q40/simplified.txt | 10 +- .../q41/simplified.txt | 6 +- .../q42/simplified.txt | 14 +-- .../q43/simplified.txt | 14 +-- .../q44/simplified.txt | 6 +- .../q45/simplified.txt | 28 ++--- .../q46/simplified.txt | 28 ++--- .../q47/simplified.txt | 20 +-- .../q48/simplified.txt | 24 ++-- .../q49/simplified.txt | 36 +++--- .../q5/simplified.txt | 52 ++++---- .../q50/simplified.txt | 26 ++-- .../q51/simplified.txt | 14 +-- .../q52/simplified.txt | 14 +-- .../q53/simplified.txt | 20 +-- .../q54/simplified.txt | 32 ++--- .../q55/simplified.txt | 14 +-- .../q56/simplified.txt | 44 +++---- .../q57/simplified.txt | 20 +-- .../q58/simplified.txt | 36 +++--- .../q59/simplified.txt | 10 +- .../q6/simplified.txt | 28 ++--- .../q60/simplified.txt | 44 +++---- .../q61/simplified.txt | 50 ++++---- .../q62/simplified.txt | 26 ++-- .../q63/simplified.txt | 20 +-- .../q64/simplified.txt | 32 ++--- .../q65/simplified.txt | 18 +-- .../q66/simplified.txt | 38 +++--- .../q67/simplified.txt | 22 ++-- .../q68/simplified.txt | 28 ++--- .../q69/simplified.txt | 22 ++-- .../q7/simplified.txt | 28 ++--- .../q70/simplified.txt | 22 ++-- .../q71/simplified.txt | 32 ++--- .../q72/simplified.txt | 50 ++++---- .../q73/simplified.txt | 26 ++-- .../q74/simplified.txt | 52 ++++---- .../q75/simplified.txt | 58 ++++----- .../q76/simplified.txt | 28 ++--- .../q77/simplified.txt | 38 +++--- .../q78/simplified.txt | 14 +-- .../q79/simplified.txt | 24 ++-- .../q8/simplified.txt | 26 ++-- .../q80/simplified.txt | 24 ++-- .../q81/simplified.txt | 24 ++-- .../q82/simplified.txt | 22 ++-- .../q83.ansi/simplified.txt | 38 +++--- .../q84/simplified.txt | 28 ++--- .../q85/simplified.txt | 40 +++--- .../q86/simplified.txt | 16 +-- .../q87/simplified.txt | 28 ++--- .../q88/simplified.txt | 104 +++++++-------- .../q89/simplified.txt | 22 ++-- .../q9/simplified.txt | 10 +- .../q90/simplified.txt | 32 ++--- .../q91/simplified.txt | 40 +++--- .../q92/simplified.txt | 18 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 8 +- .../q95/simplified.txt | 12 +- .../q96/simplified.txt | 20 +-- .../q97/simplified.txt | 10 +- .../q98/simplified.txt | 16 +-- .../q99/simplified.txt | 26 ++-- .../approved-plans-v1_4/q1/simplified.txt | 22 ++-- .../approved-plans-v1_4/q10/simplified.txt | 22 ++-- .../approved-plans-v1_4/q11/simplified.txt | 60 ++++----- .../approved-plans-v1_4/q12/simplified.txt | 18 +-- .../approved-plans-v1_4/q13/simplified.txt | 30 ++--- .../approved-plans-v1_4/q14a/simplified.txt | 56 ++++----- .../approved-plans-v1_4/q14b/simplified.txt | 58 ++++----- .../approved-plans-v1_4/q15/simplified.txt | 22 ++-- .../approved-plans-v1_4/q16/simplified.txt | 8 +- .../approved-plans-v1_4/q17/simplified.txt | 46 +++---- .../approved-plans-v1_4/q18/simplified.txt | 40 +++--- .../approved-plans-v1_4/q19/simplified.txt | 30 ++--- .../approved-plans-v1_4/q2/simplified.txt | 14 +-- .../approved-plans-v1_4/q20/simplified.txt | 18 +-- .../approved-plans-v1_4/q21/simplified.txt | 24 ++-- .../approved-plans-v1_4/q22/simplified.txt | 22 ++-- .../approved-plans-v1_4/q23a/simplified.txt | 42 +++---- .../approved-plans-v1_4/q23b/simplified.txt | 48 +++---- .../approved-plans-v1_4/q24a/simplified.txt | 14 +-- .../approved-plans-v1_4/q24b/simplified.txt | 14 +-- .../approved-plans-v1_4/q25/simplified.txt | 46 +++---- .../approved-plans-v1_4/q26/simplified.txt | 30 ++--- .../approved-plans-v1_4/q27/simplified.txt | 30 ++--- .../approved-plans-v1_4/q28/simplified.txt | 24 ++-- .../approved-plans-v1_4/q29/simplified.txt | 52 ++++---- .../approved-plans-v1_4/q3/simplified.txt | 16 +-- .../approved-plans-v1_4/q30/simplified.txt | 28 ++--- .../approved-plans-v1_4/q31/simplified.txt | 68 +++++----- .../approved-plans-v1_4/q32/simplified.txt | 20 +-- .../approved-plans-v1_4/q33/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q34/simplified.txt | 26 ++-- .../approved-plans-v1_4/q35/simplified.txt | 22 ++-- .../approved-plans-v1_4/q36/simplified.txt | 24 ++-- .../approved-plans-v1_4/q37/simplified.txt | 22 ++-- .../approved-plans-v1_4/q38/simplified.txt | 28 ++--- .../approved-plans-v1_4/q39a/simplified.txt | 38 +++--- .../approved-plans-v1_4/q39b/simplified.txt | 38 +++--- .../approved-plans-v1_4/q4/simplified.txt | 84 ++++++------- .../approved-plans-v1_4/q40/simplified.txt | 10 +- .../approved-plans-v1_4/q41/simplified.txt | 6 +- .../approved-plans-v1_4/q42/simplified.txt | 16 +-- .../approved-plans-v1_4/q43/simplified.txt | 16 +-- .../approved-plans-v1_4/q44/simplified.txt | 10 +- .../approved-plans-v1_4/q45/simplified.txt | 28 ++--- .../approved-plans-v1_4/q46/simplified.txt | 30 ++--- .../approved-plans-v1_4/q47/simplified.txt | 22 ++-- .../approved-plans-v1_4/q48/simplified.txt | 26 ++-- .../approved-plans-v1_4/q49/simplified.txt | 42 +++---- .../approved-plans-v1_4/q5/simplified.txt | 58 ++++----- .../approved-plans-v1_4/q50/simplified.txt | 28 ++--- .../approved-plans-v1_4/q51/simplified.txt | 18 +-- .../approved-plans-v1_4/q52/simplified.txt | 16 +-- .../approved-plans-v1_4/q53/simplified.txt | 22 ++-- .../approved-plans-v1_4/q54/simplified.txt | 32 ++--- .../approved-plans-v1_4/q55/simplified.txt | 16 +-- .../approved-plans-v1_4/q56/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q57/simplified.txt | 22 ++-- .../approved-plans-v1_4/q58/simplified.txt | 42 +++---- .../approved-plans-v1_4/q59/simplified.txt | 12 +- .../approved-plans-v1_4/q6/simplified.txt | 30 ++--- .../approved-plans-v1_4/q60/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q61/simplified.txt | 54 ++++---- .../approved-plans-v1_4/q62/simplified.txt | 28 ++--- .../approved-plans-v1_4/q63/simplified.txt | 22 ++-- .../approved-plans-v1_4/q64/simplified.txt | 32 ++--- .../approved-plans-v1_4/q65/simplified.txt | 22 ++-- .../approved-plans-v1_4/q66/simplified.txt | 42 +++---- .../approved-plans-v1_4/q67/simplified.txt | 24 ++-- .../approved-plans-v1_4/q68/simplified.txt | 30 ++--- .../approved-plans-v1_4/q69/simplified.txt | 22 ++-- .../approved-plans-v1_4/q7/simplified.txt | 30 ++--- .../approved-plans-v1_4/q70/simplified.txt | 24 ++-- .../approved-plans-v1_4/q71/simplified.txt | 34 ++--- .../approved-plans-v1_4/q72/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q73/simplified.txt | 26 ++-- .../approved-plans-v1_4/q74/simplified.txt | 60 ++++----- .../approved-plans-v1_4/q75/simplified.txt | 58 ++++----- .../approved-plans-v1_4/q76/simplified.txt | 30 ++--- .../approved-plans-v1_4/q77/simplified.txt | 50 ++++---- .../approved-plans-v1_4/q78/simplified.txt | 14 +-- .../approved-plans-v1_4/q79/simplified.txt | 26 ++-- .../approved-plans-v1_4/q8/simplified.txt | 26 ++-- .../approved-plans-v1_4/q80/simplified.txt | 24 ++-- .../approved-plans-v1_4/q81/simplified.txt | 28 ++--- .../approved-plans-v1_4/q82/simplified.txt | 22 ++-- .../approved-plans-v1_4/q83/simplified.txt | 44 +++---- .../approved-plans-v1_4/q84/simplified.txt | 28 ++--- .../approved-plans-v1_4/q85/simplified.txt | 42 +++---- .../approved-plans-v1_4/q86/simplified.txt | 18 +-- .../approved-plans-v1_4/q87/simplified.txt | 28 ++--- .../approved-plans-v1_4/q88/simplified.txt | 104 +++++++-------- .../approved-plans-v1_4/q89/simplified.txt | 24 ++-- .../approved-plans-v1_4/q9/simplified.txt | 20 +-- .../approved-plans-v1_4/q90/simplified.txt | 32 ++--- .../approved-plans-v1_4/q91/simplified.txt | 42 +++---- .../approved-plans-v1_4/q92/simplified.txt | 20 +-- .../approved-plans-v1_4/q93/simplified.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 8 +- .../approved-plans-v1_4/q95/simplified.txt | 12 +- .../approved-plans-v1_4/q96/simplified.txt | 20 +-- .../approved-plans-v1_4/q97/simplified.txt | 10 +- .../approved-plans-v1_4/q98/simplified.txt | 18 +-- .../approved-plans-v1_4/q99/simplified.txt | 28 ++--- .../q10a/simplified.txt | 38 +++--- .../q11/simplified.txt | 60 ++++----- .../q12/simplified.txt | 18 +-- .../q14/simplified.txt | 58 ++++----- .../q14a/simplified.txt | 62 ++++----- .../q18a/simplified.txt | 118 +++++++++--------- .../q20/simplified.txt | 18 +-- .../q22/simplified.txt | 16 +-- .../q22a/simplified.txt | 22 ++-- .../q24/simplified.txt | 14 +-- .../q27a/simplified.txt | 60 ++++----- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q35a/simplified.txt | 38 +++--- .../q36a/simplified.txt | 24 ++-- .../q47/simplified.txt | 22 ++-- .../q49/simplified.txt | 42 +++---- .../q51a/simplified.txt | 18 +-- .../q57/simplified.txt | 22 ++-- .../q5a/simplified.txt | 58 ++++----- .../q6/simplified.txt | 30 ++--- .../q64/simplified.txt | 32 ++--- .../q67a/simplified.txt | 24 ++-- .../q70a/simplified.txt | 24 ++-- .../q72/simplified.txt | 50 ++++---- .../q74/simplified.txt | 60 ++++----- .../q75/simplified.txt | 58 ++++----- .../q77a/simplified.txt | 50 ++++---- .../q78/simplified.txt | 14 +-- .../q80a/simplified.txt | 24 ++-- .../q86a/simplified.txt | 18 +-- .../q98/simplified.txt | 18 +-- .../q10a/simplified.txt | 38 +++--- .../q11/simplified.txt | 52 ++++---- .../q12/simplified.txt | 16 +-- .../q14/simplified.txt | 56 ++++----- .../q14a/simplified.txt | 60 ++++----- .../q18a/simplified.txt | 108 ++++++++-------- .../q20/simplified.txt | 16 +-- .../q22/simplified.txt | 16 +-- .../q22a/simplified.txt | 20 +-- .../q24/simplified.txt | 14 +-- .../q27a/simplified.txt | 54 ++++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q35a/simplified.txt | 36 +++--- .../q36a/simplified.txt | 22 ++-- .../q47/simplified.txt | 20 +-- .../q49/simplified.txt | 36 +++--- .../q51a/simplified.txt | 14 +-- .../q57/simplified.txt | 20 +-- .../q5a/simplified.txt | 52 ++++---- .../q6/simplified.txt | 28 ++--- .../q64/simplified.txt | 32 ++--- .../q67a/simplified.txt | 22 ++-- .../q70a/simplified.txt | 22 ++-- .../q72/simplified.txt | 50 ++++---- .../q74/simplified.txt | 52 ++++---- .../q75/simplified.txt | 58 ++++----- .../q77a/simplified.txt | 38 +++--- .../q78/simplified.txt | 14 +-- .../q80a/simplified.txt | 24 ++-- .../q86a/simplified.txt | 16 +-- .../q98/simplified.txt | 16 +-- .../approved-plans-v2_7/q10a/simplified.txt | 38 +++--- .../approved-plans-v2_7/q11/simplified.txt | 60 ++++----- .../approved-plans-v2_7/q12/simplified.txt | 18 +-- .../approved-plans-v2_7/q14/simplified.txt | 58 ++++----- .../approved-plans-v2_7/q14a/simplified.txt | 62 ++++----- .../approved-plans-v2_7/q18a/simplified.txt | 118 +++++++++--------- .../approved-plans-v2_7/q20/simplified.txt | 18 +-- .../approved-plans-v2_7/q22/simplified.txt | 16 +-- .../approved-plans-v2_7/q22a/simplified.txt | 22 ++-- .../approved-plans-v2_7/q24/simplified.txt | 14 +-- .../approved-plans-v2_7/q27a/simplified.txt | 60 ++++----- .../approved-plans-v2_7/q34/simplified.txt | 26 ++-- .../approved-plans-v2_7/q35/simplified.txt | 22 ++-- .../approved-plans-v2_7/q35a/simplified.txt | 38 +++--- .../approved-plans-v2_7/q36a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q47/simplified.txt | 22 ++-- .../approved-plans-v2_7/q49/simplified.txt | 42 +++---- .../approved-plans-v2_7/q51a/simplified.txt | 18 +-- .../approved-plans-v2_7/q57/simplified.txt | 22 ++-- .../approved-plans-v2_7/q5a/simplified.txt | 58 ++++----- .../approved-plans-v2_7/q6/simplified.txt | 30 ++--- .../approved-plans-v2_7/q64/simplified.txt | 32 ++--- .../approved-plans-v2_7/q67a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q70a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q72/simplified.txt | 50 ++++---- .../approved-plans-v2_7/q74/simplified.txt | 60 ++++----- .../approved-plans-v2_7/q75/simplified.txt | 58 ++++----- .../approved-plans-v2_7/q77a/simplified.txt | 50 ++++---- .../approved-plans-v2_7/q78/simplified.txt | 14 +-- .../approved-plans-v2_7/q80a/simplified.txt | 24 ++-- .../approved-plans-v2_7/q86a/simplified.txt | 18 +-- .../approved-plans-v2_7/q98/simplified.txt | 18 +-- 415 files changed, 6380 insertions(+), 6302 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8129c7455..3c88d6a7e 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2491,8 +2491,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff index ebc4cdb8c..cf6754416 100644 --- a/dev/diffs/3.5.1.diff +++ b/dev/diffs/3.5.1.diff @@ -2650,8 +2650,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 4031015df..744722ecf 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -2651,8 +2651,8 @@ index 5fbf379644f..32711763ec1 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index fb1f62542..c30d6b0f7 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -331,8 +331,8 @@ class CometSparkSessionExtensions CometProjectExec( nativeOp, op, - op.projectList, op.output, + op.projectList, op.child, SerializedPlan(None)) case None => @@ -343,7 +343,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometFilterExec(nativeOp, op, op.condition, op.child, SerializedPlan(None)) + CometFilterExec( + nativeOp, + op, + op.output, + op.condition, + op.child, + SerializedPlan(None)) case None => op } @@ -352,7 +358,14 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometSortExec(nativeOp, op, op.sortOrder, op.child, SerializedPlan(None)) + CometSortExec( + nativeOp, + op, + op.output, + op.outputOrdering, + op.sortOrder, + op.child, + SerializedPlan(None)) case None => op } @@ -393,12 +406,27 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometExpandExec(nativeOp, op, op.projections, op.child, SerializedPlan(None)) + CometExpandExec( + nativeOp, + op, + op.output, + op.projections, + op.child, + SerializedPlan(None)) case None => op } - case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) => + case op @ HashAggregateExec( + _, + _, + _, + groupingExprs, + aggExprs, + _, + _, + resultExpressions, + child) => val modes = aggExprs.map(_.mode).distinct if (!modes.isEmpty && modes.size != 1) { @@ -425,8 +453,10 @@ class CometSparkSessionExtensions CometHashAggregateExec( nativeOp, op, + op.output, groupingExprs, aggExprs, + resultExpressions, child.output, if (modes.nonEmpty) Some(modes.head) else None, child, @@ -446,6 +476,8 @@ class CometSparkSessionExtensions CometHashJoinExec( nativeOp, op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -478,6 +510,8 @@ class CometSparkSessionExtensions CometBroadcastHashJoinExec( nativeOp, op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -499,6 +533,8 @@ class CometSparkSessionExtensions CometSortMergeJoinExec( nativeOp, op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -535,7 +571,7 @@ class CometSparkSessionExtensions && isCometNative(child) => QueryPlanSerde.operator2Proto(c) match { case Some(nativeOp) => - val cometOp = CometCoalesceExec(c, numPartitions, child) + val cometOp = CometCoalesceExec(c, c.output, numPartitions, child) CometSinkPlaceHolder(nativeOp, c, cometOp) case None => c @@ -559,7 +595,13 @@ class CometSparkSessionExtensions QueryPlanSerde.operator2Proto(s) match { case Some(nativeOp) => val cometOp = - CometTakeOrderedAndProjectExec(s, s.limit, s.sortOrder, s.projectList, s.child) + CometTakeOrderedAndProjectExec( + s, + s.output, + s.limit, + s.sortOrder, + s.projectList, + s.child) CometSinkPlaceHolder(nativeOp, s, cometOp) case None => s @@ -580,7 +622,13 @@ class CometSparkSessionExtensions newOp match { case Some(nativeOp) => val cometOp = - CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) + CometWindowExec( + w, + w.output, + w.windowExpression, + w.partitionSpec, + w.orderSpec, + w.child) CometSinkPlaceHolder(nativeOp, w, cometOp) case None => w @@ -591,7 +639,7 @@ class CometSparkSessionExtensions u.children.forall(isCometNative) => QueryPlanSerde.operator2Proto(u) match { case Some(nativeOp) => - val cometOp = CometUnionExec(u, u.children) + val cometOp = CometUnionExec(u, u.output, u.children) CometSinkPlaceHolder(nativeOp, u, cometOp) case None => u @@ -631,7 +679,7 @@ class CometSparkSessionExtensions isSpark34Plus => // Spark 3.4+ only QueryPlanSerde.operator2Proto(b) match { case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.child) + val cometOp = CometBroadcastExchangeExec(b, b.output, b.child) CometSinkPlaceHolder(nativeOp, b, cometOp) case None => b } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index 38247b2c4..64d14a00e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -31,6 +31,7 @@ import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} @@ -60,7 +61,10 @@ import org.apache.comet.CometRuntimeException * Note that this only supports Spark 3.4 and later, because the serialization class * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later. */ -case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) +case class CometBroadcastExchangeExec( + originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val child: SparkPlan) extends BroadcastExchangeLike with ShimCometBroadcastExchangeExec { import CometBroadcastExchangeExec._ @@ -75,7 +79,7 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) override def doCanonicalize(): SparkPlan = { - CometBroadcastExchangeExec(originalPlan.canonicalized, child.canonicalized) + CometBroadcastExchangeExec(null, null, child.canonicalized) } override def runtimeStatistics: Statistics = { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala index cc635d739..b216c7d81 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -32,6 +33,7 @@ import com.google.common.base.Objects */ case class CometCoalesceExec( override val originalPlan: SparkPlan, + override val output: Seq[Attribute], numPartitions: Int, child: SparkPlan) extends CometExec diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index d2c9158ee..3effcca4f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -22,6 +22,8 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer} @@ -45,6 +47,8 @@ case class CometCollectLimitExec( child: SparkPlan) extends CometExec with UnaryExecNode { + override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 6e9bfe424..3f09e2fe2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -42,14 +42,13 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec */ case class CometTakeOrderedAndProjectExec( override val originalPlan: SparkPlan, + override val output: Seq[Attribute], limit: Int, sortOrder: Seq[SortOrder], projectList: Seq[NamedExpression], child: SparkPlan) extends CometExec with UnaryExecNode { - override def output: Seq[Attribute] = projectList.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 9685e75e1..4c10a8abb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -43,6 +43,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi */ case class CometWindowExec( override val originalPlan: SparkPlan, + override val output: Seq[Attribute], windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], @@ -52,8 +53,6 @@ case class CometWindowExec( override def nodeName: String = "CometWindowExec" - override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 8b1ba168f..1c9bf9947 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf @@ -154,7 +153,7 @@ abstract class CometNativeExec extends CometExec { /** The Comet native operator */ def nativeOp: Operator - override protected def doPrepare(): Unit = prepareSubqueries(originalPlan) + override protected def doPrepare(): Unit = prepareSubqueries(this) override lazy val metrics: Map[String, SQLMetric] = CometMetricNode.baselineMetrics(sparkContext) @@ -217,12 +216,12 @@ abstract class CometNativeExec extends CometExec { val it = new CometExecIterator(CometExec.newIterId, inputs, serializedPlanCopy, nativeMetrics) - setSubqueries(it.id, originalPlan) + setSubqueries(it.id, this) Option(TaskContext.get()).foreach { context => context.addTaskCompletionListener[Unit] { _ => it.close() - cleanSubqueries(it.id, originalPlan) + cleanSubqueries(it.id, this) } } @@ -381,11 +380,13 @@ abstract class CometNativeExec extends CometExec { */ protected def canonicalizePlans(): CometNativeExec = { def transform(arg: Any): AnyRef = arg match { - case sparkPlan: SparkPlan if !sparkPlan.isInstanceOf[CometNativeExec] => + case sparkPlan: SparkPlan + if !sparkPlan.isInstanceOf[CometNativeExec] && + children.forall(_ != sparkPlan) => // Different to Spark, Comet native query node might have a Spark plan as Product element. // We need to canonicalize the Spark plan. But it cannot be another Comet native query node, // otherwise it will cause recursive canonicalization. - sparkPlan.canonicalized + null case other: AnyRef => other case null => null } @@ -412,8 +413,8 @@ case class SerializedPlan(plan: Option[Array[Byte]]) { case class CometProjectExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, - projectList: Seq[NamedExpression], override val output: Seq[Attribute], + projectList: Seq[NamedExpression], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec @@ -443,18 +444,20 @@ case class CometProjectExec( case class CometFilterExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], condition: Expression, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, condition, child) + Iterator(output, condition, child) override def equals(obj: Any): Boolean = { obj match { @@ -480,6 +483,8 @@ case class CometFilterExec( case class CometSortExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], sortOrder: Seq[SortOrder], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -491,7 +496,7 @@ case class CometSortExec( this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, sortOrder, child) + Iterator(output, sortOrder, child) override def equals(obj: Any): Boolean = { obj match { @@ -520,7 +525,9 @@ case class CometLocalLimitExec( override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -550,7 +557,9 @@ case class CometGlobalLimitExec( override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -573,10 +582,13 @@ case class CometGlobalLimitExec( case class CometExpandExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], projections: Seq[Seq[Expression]], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def outputPartitioning: Partitioning = UnknownPartitioning(0) + override def producedAttributes: AttributeSet = outputSet override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -600,7 +612,10 @@ case class CometExpandExec( override lazy val metrics: Map[String, SQLMetric] = Map.empty } -case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan]) +case class CometUnionExec( + override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + children: Seq[SparkPlan]) extends CometExec { override def doExecuteColumnar(): RDD[ColumnarBatch] = { sparkContext.union(children.map(_.executeColumnar())) @@ -634,8 +649,10 @@ case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[Sp case class CometHashAggregateExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression], input: Seq[Attribute], mode: Option[AggregateMode], child: SparkPlan, @@ -674,13 +691,14 @@ case class CometHashAggregateExec( override def hashCode(): Int = Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child) - override protected def outputExpressions: Seq[NamedExpression] = - originalPlan.asInstanceOf[HashAggregateExec].resultExpressions + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions } case class CometHashJoinExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -733,6 +751,8 @@ case class CometHashJoinExec( case class CometBroadcastHashJoinExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -867,6 +887,8 @@ case class CometBroadcastHashJoinExec( case class CometSortMergeJoinExec( override val nativeOp: Operator, override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt index dc0fb18f3..d6f75837b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt index f24e34c36..a80e0ca71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt index 0a66507e1..232d65d72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt @@ -30,9 +30,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -40,9 +40,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -69,7 +69,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt index c4ea8fe24..537cba446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt index ef965d7d0..4dc4bcaf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt @@ -17,15 +17,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt index 7cddbc640..030dfc4d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -58,15 +58,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt index ac7b2c7e0..11f3a0ff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt index ecae29c78..2313467ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt index 26b5570b7..43bad5d70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -47,8 +47,8 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt index fe6b3a625..240d88bb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -15,23 +15,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -44,23 +44,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -74,13 +74,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -95,13 +95,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 4bdb7ae0f..1640fa3b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt index ef54c7add..cc52bb323 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -7,16 +7,16 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt index 72aa4a054..d2ea2d830 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt @@ -20,19 +20,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt index 73badab8d..cd4817202 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt @@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 @@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 @@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #6 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 @@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt index b0fd9ebf1..a34cd9c1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -8,12 +8,12 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt index 5c9726a22..4a634b069 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt index 561ad3d4f..c49bfc6c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt @@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag InputAdapter CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt index 852b61d1c..60caecf91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -8,31 +8,31 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt index 334c6f0ea..535c61622 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -15,14 +15,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index dd1edb95f..5bfd9a907 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt index c6058d18f..2c8892b68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt @@ -9,27 +9,27 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt index d89167ec8..eb25d6d3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt @@ -9,12 +9,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] InputAdapter CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt index 1ed007bc7..412f27ffb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -83,10 +83,10 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -109,11 +109,11 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt index 09a2bf7ce..d784ed47e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -84,10 +84,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -110,11 +110,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index 41efffcb4..6335e50ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt index 67f335c66..b98e61704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt index 74c8a1ca2..4d4c824bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] InputAdapter CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt index a7a3f9537..2136886fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -35,7 +35,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -53,7 +53,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -71,7 +71,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -89,7 +89,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -107,5 +107,5 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt index dbdfc8572..ee1f435d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt index 3dc9ede25..36bb17a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt index d5b8077ba..c4e59b49a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt @@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt index c004db9b7..9040179da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -19,22 +19,22 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -48,20 +48,20 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -75,20 +75,20 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -102,10 +102,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -121,10 +121,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -140,10 +140,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index 8cc26d8f4..b6ba1fbab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -38,13 +38,13 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt index 84143e015..bf0ef7ae4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 5bdf60081..58d617430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt index ff321c523..d16db5c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -14,12 +14,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -52,12 +52,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt index ff321c523..d16db5c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -14,12 +14,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -52,12 +52,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt index 0918da575..a412a54d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt @@ -18,23 +18,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -47,23 +47,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -77,13 +77,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -98,13 +98,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -120,13 +120,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -141,13 +141,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt index e3967a972..d6e3733ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt index 63945612a..f21d846c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed ColumnarToRow InputAdapter CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt index 35a3e9efa..12c8e0af9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt @@ -35,7 +35,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -45,7 +45,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (10) @@ -75,7 +75,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt index 718557c10..9ce3a98eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 66eb5a4cb..1db8e7528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt index 30f9822ab..23e083535 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt @@ -7,14 +7,14 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index d1480ef10..bd3b21cdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt index 3dc3cca6c..ceb3e9213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -52,21 +52,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -77,23 +77,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt index cd8a8ff7f..aa3f7f2cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt @@ -8,17 +8,17 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index 9ab7f18e7..f19312ca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -32,8 +32,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -69,8 +69,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt index d0b9da882..d7661b7be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index a04777412..b90cb42d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -17,16 +17,16 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt index 32e9a92a7..a8df48770 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [(d_month_seq + 1)] #3 ReusedSubquery [(d_month_seq + 3)] #4 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] @@ -85,7 +85,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -97,7 +97,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter BroadcastExchange #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt index a404b9b37..2461ab830 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [ext_price,brand_id,brand] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt index a5cb59e57..c4ce35e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index ee91816c9..b64c2efdc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt index 959d98ba9..0f5a783b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,12 +24,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] Subquery #2 @@ -37,15 +37,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -60,10 +60,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -80,10 +80,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt index 55cb85ddb..58bae46d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -14,18 +14,18 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt index a7c65f50a..febbfd32f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 @@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 @@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state] HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometFilter [i_category] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index cf226bb14..51025cf34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt index db13f94bd..8d39d9e4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -9,18 +9,18 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -64,16 +64,16 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt index b25008e64..b9e35e282 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt @@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ColumnarToRow InputAdapter CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index a8eb9d7c6..e84d3c2a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -17,16 +17,16 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt index 1e34ce22f..84d23af42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -22,8 +22,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -61,8 +61,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt index 7d6c85af3..880c38bbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt @@ -16,35 +16,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -55,14 +55,14 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt index 0386feff9..46d9cf4ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt @@ -20,12 +20,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt index 1ecde47e8..78e1b1cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt index 0429448f5..a27daca36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index c962534aa..4a6dffc08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -60,13 +60,13 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index c732d903f..a4be53684 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -11,17 +11,17 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt index ea253444c..a7aa864ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt index 6f8c28004..cfcf6a99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -7,32 +7,32 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] ColumnarToRow InputAdapter - CometUnion + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt index 2989455d9..91786374e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -19,10 +19,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -84,7 +84,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -99,14 +99,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -120,10 +120,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt index 5fcbcfed3..f2883e450 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt index 3daee59f8..991c00de9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt @@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt index 7b7e907ee..8cd3f3f45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt @@ -12,10 +12,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -54,10 +54,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -73,10 +73,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt index ad94ef82c..0d350bd59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt @@ -8,21 +8,21 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index ea5dce5b6..e4d028f2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -17,10 +17,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 52e066fe7..2bfff90d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -17,15 +17,15 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt index fdd3bd293..581739cc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt @@ -11,7 +11,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -41,7 +41,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -56,7 +56,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -71,7 +71,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt index 7e9b7b8ba..84f1190d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -11,21 +11,21 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index bcc1ef1b5..aa24a049b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -38,13 +38,13 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 84d277886..a92d99be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -20,10 +20,10 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt index da4ef22e0..25ff3a2fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt @@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , ColumnarToRow InputAdapter CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 000160518..238f9b702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 50472502e..60cad82dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 701a90912..f713176f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -27,9 +27,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -37,9 +37,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -57,7 +57,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index c4ea8fe24..537cba446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index b529cb5ff..c37569802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,15 +30,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index f4a03a2d2..f95e8d040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -57,15 +57,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index f80554fb5..e576a079b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index ecae29c78..2313467ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index dd66c9582..1428c7ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index aeb5b67da..5838786d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -20,14 +20,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index 74305dd6e..e76a524db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -65,16 +65,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt index 4b9e60e18..d7c6ef3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 3d0576653..07c91b94d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt index 73badab8d..cd4817202 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt @@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 @@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 @@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #6 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 @@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt index 6e0ac9858..3dbd7bc33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt index 9f43cdc72..509d4be06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt @@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -60,25 +60,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -91,26 +91,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -123,26 +123,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -155,18 +155,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index da567f687..cd6bcfd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt index db81d1ba5..fc4778a22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt index b09f5af82..50e246e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -49,21 +49,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -75,19 +75,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt index a95475f7b..79ead0464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt @@ -8,17 +8,17 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index 485fdc3e2..59c22c335 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index a480db5ea..61f4976b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index d1480ef10..bd3b21cdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index 892a3fc78..47ad5e34f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -44,8 +44,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -98,8 +98,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index b038aedf0..daeafbf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt index 34ad132c0..c3b00538b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -59,21 +59,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -84,25 +84,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt index a7c65f50a..febbfd32f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 @@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 @@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state] HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometFilter [i_category] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt index f68ac8dfc..cc9627e89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index a7c639a49..67f4841b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -67,14 +67,14 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt index a2cb15b4e..4360a9555 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt index 141e72805..db4e10247 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -26,10 +26,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -57,10 +57,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -79,7 +79,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -91,7 +91,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -106,14 +106,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -127,10 +127,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index 7edc6b80b..8996119f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -24,10 +24,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index 2dd86f85b..3fc9eb261 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -19,10 +19,10 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index dd66c9582..1428c7ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #21 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index a3a99e312..dfa9ae66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,17 +30,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 9e72edbb4..feca96b18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -64,16 +64,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] From 15e7baa5d1f76a1a0720e4f848cd01a14c09e147 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 9 Jul 2024 15:07:01 -0600 Subject: [PATCH 03/14] feat: Upgrade to DataFusion 40.0.0-rc1 (#644) * Partial upgrade to DataFusion 40.0.0 * fix * implement more udaf * update bitwise agg * add func names * remove unused imports * remove arrow-string dep * fix copy and paste error * use 40.0.0-rc1 and temporarily ignore failing test * clippy * fall back to Spark for count windows aggregate * address feedback --- native/Cargo.lock | 128 ++++++++++-------- native/core/Cargo.toml | 24 ++-- .../execution/datafusion/expressions/abs.rs | 2 +- .../execution/datafusion/expressions/avg.rs | 2 +- .../execution/datafusion/operators/expand.rs | 4 + .../core/src/execution/datafusion/planner.rs | 119 ++++++++++++---- .../execution/datafusion/shuffle_writer.rs | 4 + native/core/src/execution/operators/copy.rs | 4 + native/core/src/execution/operators/scan.rs | 4 + .../apache/comet/serde/QueryPlanSerde.scala | 5 +- .../apache/comet/exec/CometExecSuite.scala | 2 +- 11 files changed, 202 insertions(+), 96 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 6136e0339..df1828ee0 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -114,8 +114,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6127ea5e585a12ec9f742232442828ebaf264dfa5eefdd71282376c599562b77" dependencies = [ "arrow-arith", "arrow-array", @@ -134,8 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7add7f39210b7d726e2a8efc0083e7bf06e8f2d15bdb4896b564dce4410fbf5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -148,8 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81c16ec702d3898c2f5cfdc148443c6cd7dbe5bac28399859eb0a3d38f072827" dependencies = [ "ahash", "arrow-buffer", @@ -164,8 +167,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cae6970bab043c4fbc10aee1660ceb5b306d0c42c8cc5f6ae564efcd9759b663" dependencies = [ "bytes", "half", @@ -174,8 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c7ef44f26ef4f8edc392a048324ed5d757ad09135eff6d5509e6450d39e0398" dependencies = [ "arrow-array", "arrow-buffer", @@ -194,8 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f843490bd258c5182b66e888161bb6f198f49f3792f7c7f98198b924ae0f564" dependencies = [ "arrow-array", "arrow-buffer", @@ -212,8 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a769666ffac256dd301006faca1ca553d0ae7cffcf4cd07095f73f95eb226514" dependencies = [ "arrow-buffer", "arrow-schema", @@ -223,8 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbf9c3fb57390a1af0b7bb3b5558c1ee1f63905f3eccf49ae7676a8d1e6e5a72" dependencies = [ "arrow-array", "arrow-buffer", @@ -237,8 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "654e7f3724176b66ddfacba31af397c48e106fbe4d281c8144e7d237df5acfd7" dependencies = [ "arrow-array", "arrow-buffer", @@ -256,8 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8008370e624e8e3c68174faaf793540287106cfda8ad1da862fdc53d8e096b4" dependencies = [ "arrow-array", "arrow-buffer", @@ -270,8 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca5e3a6b7fda8d9fe03f3b18a2d946354ea7f3c8e4076dbdb502ad50d9d44824" dependencies = [ "ahash", "arrow-array", @@ -284,16 +295,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dab1c12b40e29d9f3b699e0203c2a73ba558444c05e388a4377208f8f9c97eee" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e80159088ffe8c48965cb9b1a7c968b2729f29f37363df7eca177fc3281fe7c3" dependencies = [ "ahash", "arrow-array", @@ -305,8 +318,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fd04a6ea7de183648edbcb7a6dd925bbd04c210895f6384c780e27a9b54afcd" dependencies = [ "arrow-array", "arrow-buffer", @@ -790,8 +804,8 @@ dependencies = [ [[package]] name = "datafusion" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -890,8 +904,8 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -910,16 +924,16 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "tokio", ] [[package]] name = "datafusion-execution" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "chrono", @@ -938,8 +952,8 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -956,8 +970,8 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "base64", @@ -967,7 +981,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "hashbrown", "hex", "itertools 0.12.1", @@ -982,8 +995,8 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -999,8 +1012,8 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "async-trait", @@ -1012,13 +1025,14 @@ dependencies = [ "indexmap", "itertools 0.12.1", "log", + "paste", "regex-syntax", ] [[package]] name = "datafusion-physical-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1032,7 +1046,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate", "datafusion-physical-expr-common", "half", "hashbrown", @@ -1047,19 +1060,21 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ + "ahash", "arrow", "datafusion-common", "datafusion-expr", + "hashbrown", "rand", ] [[package]] name = "datafusion-physical-plan" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1091,8 +1106,8 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "arrow-array", @@ -2015,8 +2030,9 @@ dependencies = [ [[package]] name = "parquet" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f22ba0d95db56dde8685e3fadcb915cdaadda31ab8abbe3ff7f0ad1ef333267" dependencies = [ "ahash", "bytes", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index bd0a3d5e4..160db2949 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -32,12 +32,12 @@ include = [ ] [dependencies] -arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } +arrow = { version = "52.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "52.1.0" } +arrow-buffer = { version = "52.1.0" } +arrow-data = { version = "52.1.0" } +arrow-schema = { version = "52.1.0" } +parquet = { version = "52.1.0", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } @@ -64,12 +64,12 @@ itertools = "0.11.0" chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } paste = "1.0.14" -datafusion-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1" } -datafusion = { default-features = false, git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["crypto_expressions"] } -datafusion-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } -datafusion-physical-expr-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } -datafusion-physical-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1" } +datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["crypto_expressions"] } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } unicode-segmentation = "^1.10.1" once_cell = "1.18.0" regex = "1.9.6" diff --git a/native/core/src/execution/datafusion/expressions/abs.rs b/native/core/src/execution/datafusion/expressions/abs.rs index 4eb8c7c1e..a037e5cbc 100644 --- a/native/core/src/execution/datafusion/expressions/abs.rs +++ b/native/core/src/execution/datafusion/expressions/abs.rs @@ -37,7 +37,7 @@ impl CometAbsFunc { pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result { if let EvalMode::Legacy | EvalMode::Ansi = eval_mode { Ok(Self { - inner_abs_func: math::abs().inner(), + inner_abs_func: math::abs().inner().clone(), eval_mode, data_type_name, }) diff --git a/native/core/src/execution/datafusion/expressions/avg.rs b/native/core/src/execution/datafusion/expressions/avg.rs index 1ff276e5d..3c8865bd1 100644 --- a/native/core/src/execution/datafusion/expressions/avg.rs +++ b/native/core/src/execution/datafusion/expressions/avg.rs @@ -47,7 +47,7 @@ pub struct Avg { impl Avg { /// Create a new AVG aggregate function pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self { - let result_data_type = avg_return_type(&data_type).unwrap(); + let result_data_type = avg_return_type("avg", &data_type).unwrap(); Self { name: name.into(), diff --git a/native/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/datafusion/operators/expand.rs index 5285dfb46..67171212f 100644 --- a/native/core/src/execution/datafusion/operators/expand.rs +++ b/native/core/src/execution/datafusion/operators/expand.rs @@ -126,6 +126,10 @@ impl ExecutionPlan for CometExpandExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "CometExpandExec" + } } pub struct ExpandStream { diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 40515c0c4..360380400 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -20,19 +20,22 @@ use std::{collections::HashMap, sync::Arc}; use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use datafusion::functions_aggregate::bit_and_or_xor::{bit_and_udaf, bit_or_udaf, bit_xor_udaf}; +use datafusion::functions_aggregate::count::count_udaf; +use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::physical_plan::windows::BoundedWindowAggExec; use datafusion::physical_plan::InputOrderMode; use datafusion::{ arrow::{compute::SortOptions, datatypes::SchemaRef}, common::DataFusionError, execution::FunctionRegistry, + functions_aggregate::first_last::{FirstValue, LastValue}, logical_expr::Operator as DataFusionOperator, physical_expr::{ execution_props::ExecutionProps, expressions::{ - in_list, BinaryExpr, BitAnd, BitOr, BitXor, CaseExpr, CastExpr, Column, Count, - FirstValue, IsNotNullExpr, IsNullExpr, LastValue, Literal as DataFusionLiteral, Max, - Min, NotExpr, Sum, + in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, + Literal as DataFusionLiteral, Max, Min, NotExpr, }, AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }, @@ -647,7 +650,7 @@ impl PhysicalPlanner { let left = self.create_expr(left, input_schema.clone())?; let right = self.create_expr(right, input_schema.clone())?; match ( - op, + &op, left.data_type(&input_schema), right.data_type(&input_schema), ) { @@ -1208,11 +1211,19 @@ impl PhysicalPlanner { .iter() .map(|child| self.create_expr(child, schema.clone())) .collect::, _>>()?; - Ok(Arc::new(Count::new_with_multiple_exprs( - children, + + create_aggregate_expr( + &count_udaf(), + &children, + &[], + &[], + &[], + schema.as_ref(), "count", - DataType::Int64, - ))) + false, + false, + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) } AggExprStruct::Min(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; @@ -1236,7 +1247,18 @@ impl PhysicalPlanner { // cast to the result data type of SUM if necessary, we should not expect // a cast failure since it should have already been checked at Spark side let child = Arc::new(CastExpr::new(child, datatype.clone(), None)); - Ok(Arc::new(Sum::new(child, "sum", datatype))) + create_aggregate_expr( + &sum_udaf(), + &[child], + &[], + &[], + &[], + schema.as_ref(), + "sum", + false, + false, + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) } } } @@ -1263,31 +1285,79 @@ impl PhysicalPlanner { AggExprStruct::First(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - - create_aggregate_expr(&func, &[child], &[], &[], &schema, "first", false, false) - .map_err(|e| e.into()) + create_aggregate_expr( + &func, + &[child], + &[], + &[], + &[], + &schema, + "first", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::Last(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); - - create_aggregate_expr(&func, &[child], &[], &[], &schema, "last", false, false) - .map_err(|e| e.into()) + create_aggregate_expr( + &func, + &[child], + &[], + &[], + &[], + &schema, + "last", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitAndAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitAnd::new(child, "bit_and", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_and_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_and", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitOrAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitOr::new(child, "bit_or", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_or_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_or", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitXorAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitXor::new(child, "bit_xor", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_xor_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_xor", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::Covariance(expr) => { let child1 = self.create_expr(expr.child1.as_ref().unwrap(), schema.clone())?; @@ -1483,6 +1553,7 @@ impl PhysicalPlanner { &window_func, window_func_name, &window_args, + &[], partition_by, sort_exprs, window_frame.into(), diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs index 5afc9a53e..6e59ce53a 100644 --- a/native/core/src/execution/datafusion/shuffle_writer.rs +++ b/native/core/src/execution/datafusion/shuffle_writer.rs @@ -160,6 +160,10 @@ impl ExecutionPlan for ShuffleWriterExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "ShuffleWriterExec" + } } impl ShuffleWriterExec { diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index d011b3cb2..68c91aafc 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -126,6 +126,10 @@ impl ExecutionPlan for CopyExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "CopyExec" + } } struct CopyStream { diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index de5328210..68dd773cf 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -270,6 +270,10 @@ impl ExecutionPlan for ScanExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "ScanExec" + } } impl DisplayAs for ScanExec { diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 65de37c83..da534b02c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -208,7 +208,10 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim expr match { case agg: AggregateExpression => agg.aggregateFunction match { - case _: Min | _: Max | _: Count => + // TODO add support for Count (this was removed when upgrading + // to DataFusion 40 because it is no longer a built-in window function) + // https://github.com/apache/datafusion-comet/issues/645 + case _: Min | _: Max => Some(agg) case _ => withInfo(windowExpr, "Unsupported aggregate", expr) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 9cc4e7f78..e657af9b9 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -1438,7 +1438,7 @@ class CometExecSuite extends CometTestBase { SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { withParquetTable((0 until 10).map(i => (i, 10 - i)), "t1") { // TODO: test nulls val aggregateFunctions = - List("COUNT(_1)", "MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates + List("MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates aggregateFunctions.foreach { function => val queries = Seq( From 33706125b8c7a7f347865c7fb38fede6aceb97e9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Jul 2024 12:10:25 -0700 Subject: [PATCH 04/14] feat: Use unified allocator for execution iterators (#613) * feat: Use unified allocator for execution iterators * Disable CometTakeOrderedAndProjectExec * Add comment * Increase heap memory * Enable CometTakeOrderedAndProjectExec * More * More * Reduce heap memory * Run sort merge join TPCDS with -e for debugging * Add -X flag * Disable q72 and q72-v2.7 * Update .github/workflows/benchmark.yml --- .../main/scala/org/apache/comet/package.scala | 11 ++++ .../org/apache/comet/vector/NativeUtil.scala | 6 +- .../apache/comet/vector/StreamReader.scala | 12 ++-- .../spark/sql/CometTPCDSQuerySuite.scala | 56 +++++++++++++++---- 4 files changed, 65 insertions(+), 20 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/package.scala b/common/src/main/scala/org/apache/comet/package.scala index c9aca7538..f44139ba6 100644 --- a/common/src/main/scala/org/apache/comet/package.scala +++ b/common/src/main/scala/org/apache/comet/package.scala @@ -21,8 +21,19 @@ package org.apache import java.util.Properties +import org.apache.arrow.memory.RootAllocator + package object comet { + /** + * The root allocator for Comet execution. Because Arrow Java memory management is based on + * reference counting, exposed arrays increase the reference count of the underlying buffers. + * Until the reference count is zero, the memory will not be released. If the consumer side is + * finished later than the close of the allocator, the allocator will think the memory is + * leaked. To avoid this, we use a single allocator for the whole execution process. + */ + val CometArrowAllocator = new RootAllocator(Long.MaxValue) + /** * Provides access to build information about the Comet libraries. This will be used by the * benchmarking software to provide the source revision and repository. In addition, the build diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala index 595c0a427..89f79c9cd 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -22,18 +22,18 @@ package org.apache.comet.vector import scala.collection.mutable import org.apache.arrow.c.{ArrowArray, ArrowImporter, ArrowSchema, CDataDictionaryProvider, Data} -import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.dictionary.DictionaryProvider import org.apache.spark.SparkException import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.comet.CometArrowAllocator + class NativeUtil { import Utils._ - private val allocator = new RootAllocator(Long.MaxValue) - .newChildAllocator(this.getClass.getSimpleName, 0, Long.MaxValue) + private val allocator = CometArrowAllocator private val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider private val importer = new ArrowImporter(allocator) diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala index 4a08f0521..b8106a96e 100644 --- a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala +++ b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala @@ -21,20 +21,20 @@ package org.apache.comet.vector import java.nio.channels.ReadableByteChannel -import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.{ArrowStreamReader, ReadChannel} import org.apache.arrow.vector.ipc.message.MessageChannelReader import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.comet.CometArrowAllocator + /** * A reader that consumes Arrow data from an input channel, and produces Comet batches. */ case class StreamReader(channel: ReadableByteChannel, source: String) extends AutoCloseable { - private var allocator = new RootAllocator(Long.MaxValue) - .newChildAllocator(s"${this.getClass.getSimpleName}/$source", 0, Long.MaxValue) - private val channelReader = new MessageChannelReader(new ReadChannel(channel), allocator) - private var arrowReader = new ArrowStreamReader(channelReader, allocator) + private val channelReader = + new MessageChannelReader(new ReadChannel(channel), CometArrowAllocator) + private var arrowReader = new ArrowStreamReader(channelReader, CometArrowAllocator) private var root = arrowReader.getVectorSchemaRoot def nextBatch(): Option[ColumnarBatch] = { @@ -53,11 +53,9 @@ case class StreamReader(channel: ReadableByteChannel, source: String) extends Au if (root != null) { arrowReader.close() root.close() - allocator.close() arrowReader = null root = null - allocator = null } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala index 3e0f64522..6eeb7e334 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -27,10 +27,6 @@ import org.apache.comet.CometConf class CometTPCDSQuerySuite extends { - override val excludedTpcdsQueries: Set[String] = Set() - - // This is private in `TPCDSBase` and `excludedTpcdsQueries` is private too. - // So we cannot override `excludedTpcdsQueries` to exclude the queries. val tpcdsAllQueries: Seq[String] = Seq( "q1", "q2", @@ -112,7 +108,9 @@ class CometTPCDSQuerySuite "q69", "q70", "q71", - "q72", + // TODO: unknown failure (seems memory usage over Github action runner) in CI with q72 in + // https://github.com/apache/datafusion-comet/pull/613. + // "q72", "q73", "q74", "q75", @@ -141,9 +139,45 @@ class CometTPCDSQuerySuite "q98", "q99") - // TODO: enable the 3 queries after fixing the issues #1358. - override val tpcdsQueries: Seq[String] = - tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains) + val tpcdsAllQueriesV2_7_0: Seq[String] = Seq( + "q5a", + "q6", + "q10a", + "q11", + "q12", + "q14", + "q14a", + "q18a", + "q20", + "q22", + "q22a", + "q24", + "q27a", + "q34", + "q35", + "q35a", + "q36a", + "q47", + "q49", + "q51a", + "q57", + "q64", + "q67a", + "q70a", + // TODO: unknown failure (seems memory usage over Github action runner) in CI with q72-v2.7 + // in https://github.com/apache/datafusion-comet/pull/613. + // "q72", + "q74", + "q75", + "q77a", + "q78", + "q80a", + "q86a", + "q98") + + override val tpcdsQueries: Seq[String] = tpcdsAllQueries + + override val tpcdsQueriesV2_7_0: Seq[String] = tpcdsAllQueriesV2_7_0 } with CometTPCDSQueryTestSuite with ShimCometTPCDSQuerySuite { @@ -157,9 +191,11 @@ class CometTPCDSQuerySuite conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") - conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "20g") + conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "15g") + conf.set(CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key, "true") + conf.set("spark.sql.adaptive.coalescePartitions.enabled", "true") conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") - conf.set(MEMORY_OFFHEAP_SIZE.key, "20g") + conf.set(MEMORY_OFFHEAP_SIZE.key, "15g") conf } From fdd2c4f8a088b2ffedf203a6e7898f90273a0021 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Wed, 10 Jul 2024 12:10:59 -0700 Subject: [PATCH 05/14] fix: Tag ignored tests that require SubqueryBroadcastExec (#647) --- dev/diffs/4.0.0-preview1.diff | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 744722ecf..abd6e9a04 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -415,7 +415,7 @@ index 16a493b5290..3f0b70e2d59 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index 2c24cc7d570..50a2ce86117 100644 +index 2c24cc7d570..d46dc5e138a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -442,7 +442,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("partition pruning in broadcast hash joins with aliases") { + test("partition pruning in broadcast hash joins with aliases", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { Given("alias with simple join condition, using attribute names only") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( @@ -452,7 +452,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("partition pruning in broadcast hash joins") { + test("partition pruning in broadcast hash joins", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { Given("disable broadcast pruning and disable subquery duplication") withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", @@ -462,7 +462,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("different broadcast subqueries with identical children") { + test("different broadcast subqueries with identical children", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( @@ -492,7 +492,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-32817: DPP throws error when the broadcast side is empty") { + test("SPARK-32817: DPP throws error when the broadcast side is empty", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", @@ -502,7 +502,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning") { + test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ @@ -521,7 +521,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") { + test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withTable("duplicate_keys") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US")) @@ -531,7 +531,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty") { + test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ @@ -541,7 +541,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-39217: Makes DPP support the pruning side has Union") { + test("SPARK-39217: Makes DPP support the pruning side has Union", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ @@ -1159,7 +1159,7 @@ index 15de4c5cc5b..6a85dfb6883 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index b5bac8079c4..544c1ddc697 100644 +index b5bac8079c4..a3731888e12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,8 @@ @@ -1190,7 +1190,7 @@ index b5bac8079c4..544c1ddc697 100644 - test("join with ordering requirement") { + test("join with ordering requirement", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { val query = "select * from (select key, a, c, b from testView) as t1 join " + "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" assertProjectExec(query, 2, 2) @@ -2590,7 +2590,7 @@ index af07aceaed1..ed0b5e6d9be 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index 5fbf379644f..32711763ec1 100644 +index 5fbf379644f..6153046a787 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest From 10347b9c7888f38b9a2d619b36abf28bd4fec3d4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 10 Jul 2024 13:32:40 -0600 Subject: [PATCH 06/14] feat: Create new `datafusion-comet-spark-expr` crate containing Spark-compatible DataFusion expressions (#638) * convert into workspace project * update GitHub actions * update Makefile * fix regression * update target path * update protobuf path in pom.xml * update more paths * add new datafusion-comet-expr crate * rename CometAbsFunc to Abs and add documentation * fix error message * improve error handling * update crate description * remove unused dep * address feedback * finish renaming crate * update README for datafusion-spark-expr * rename crate to datafusion-comet-spark-expr --- native/Cargo.lock | 74 +- native/Cargo.toml | 24 +- native/core/Cargo.lock | 3301 ----------------- native/core/Cargo.toml | 34 +- .../execution/datafusion/expressions/mod.rs | 23 +- .../core/src/execution/datafusion/planner.rs | 17 +- native/spark-expr/Cargo.toml | 38 + native/spark-expr/README.md | 23 + .../expressions => spark-expr/src}/abs.rs | 31 +- native/spark-expr/src/lib.rs | 56 + 10 files changed, 228 insertions(+), 3393 deletions(-) delete mode 100644 native/core/Cargo.lock create mode 100644 native/spark-expr/Cargo.toml create mode 100644 native/spark-expr/README.md rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/abs.rs (81%) create mode 100644 native/spark-expr/src/lib.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index df1828ee0..c0f22fa1a 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -341,13 +341,13 @@ checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -482,9 +482,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.104" +version = "1.0.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74b6a57f98764a267ff415d50a25e6e166f3831a5071af4995296ea97d210490" +checksum = "066fce287b1d4eafef758e89e09d724a24808a9196fe9756b8ca90e86d0719a2" dependencies = [ "jobserver", "libc", @@ -866,9 +866,9 @@ dependencies = [ "crc32fast", "criterion", "datafusion", + "datafusion-comet-spark-expr", "datafusion-common", "datafusion-expr", - "datafusion-functions", "datafusion-physical-expr", "datafusion-physical-expr-common", "flate2", @@ -902,6 +902,17 @@ dependencies = [ "zstd", ] +[[package]] +name = "datafusion-comet-spark-expr" +version = "0.1.0" +dependencies = [ + "arrow", + "arrow-schema", + "datafusion", + "datafusion-common", + "datafusion-functions", +] + [[package]] name = "datafusion-common" version = "40.0.0" @@ -1300,7 +1311,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -1992,9 +2003,9 @@ checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "oorandom" -version = "11.1.3" +version = "11.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" [[package]] name = "ordered-float" @@ -2030,9 +2041,8 @@ dependencies = [ [[package]] name = "parquet" -version = "52.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f22ba0d95db56dde8685e3fadcb915cdaadda31ab8abbe3ff7f0ad1ef333267" +version = "52.0.0" +source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" dependencies = [ "ahash", "bytes", @@ -2261,7 +2271,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2382,9 +2392,9 @@ checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" [[package]] name = "rgb" -version = "0.8.40" +version = "0.8.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7439be6844e40133eda024efd85bf07f59d0dd2f59b10c00dd6cfb92cc5c741" +checksum = "1aee83dc281d5a3200d37b299acd13b81066ea126a7f16f0eae70fc9aed241d9" dependencies = [ "bytemuck", ] @@ -2458,9 +2468,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12" dependencies = [ "serde_derive", ] @@ -2477,13 +2487,13 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2594,7 +2604,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2634,7 +2644,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2679,9 +2689,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.68" +version = "2.0.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "901fa70d88b9d6c98022e23b4136f9f3e54e4662c3bc1bd1d84a42a9a0f0c1e9" +checksum = "2f0209b68b3613b093e0ec905354eccaedcfe83b8cb37cbdeae64026c3064c16" dependencies = [ "proc-macro2", "quote", @@ -2717,7 +2727,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2762,9 +2772,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce6b6a2fb3a985e99cebfaefa9faa3024743da73304ca1c683a36429613d3d22" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -2796,7 +2806,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2818,7 +2828,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2967,7 +2977,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-shared", ] @@ -2989,7 +2999,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3218,7 +3228,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index 79d393fdb..13860fbdf 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -16,7 +16,8 @@ # under the License. [workspace] -members = ["core"] +members = ["core", "spark-expr"] +resolver = "2" [workspace.package] version = "0.1.0" @@ -30,3 +31,24 @@ edition = "2021" # Comet uses the same minimum Rust version as DataFusion rust-version = "1.75" + +[workspace.dependencies] +arrow = { version = "52.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "52.1.0" } +arrow-buffer = { version = "52.1.0" } +arrow-data = { version = "52.1.0" } +arrow-schema = { version = "52.1.0" } +parquet = { version = "52.1.0", default-features = false, features = ["experimental"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1" } +datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["crypto_expressions"] } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } + +[profile.release] +debug = true +overflow-checks = false +lto = "thin" +codegen-units = 1 +strip = "debuginfo" diff --git a/native/core/Cargo.lock b/native/core/Cargo.lock deleted file mode 100644 index 71fe6eb2e..000000000 --- a/native/core/Cargo.lock +++ /dev/null @@ -1,3301 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "addr2line" -version = "0.22.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" -dependencies = [ - "gimli", -] - -[[package]] -name = "adler" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" - -[[package]] -name = "ahash" -version = "0.8.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" -dependencies = [ - "cfg-if", - "const-random", - "getrandom", - "once_cell", - "version_check", - "zerocopy", -] - -[[package]] -name = "aho-corasick" -version = "1.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" -dependencies = [ - "memchr", -] - -[[package]] -name = "alloc-no-stdlib" -version = "2.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" - -[[package]] -name = "alloc-stdlib" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" -dependencies = [ - "alloc-no-stdlib", -] - -[[package]] -name = "allocator-api2" -version = "0.2.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" - -[[package]] -name = "android-tzdata" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" - -[[package]] -name = "android_system_properties" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" -dependencies = [ - "libc", -] - -[[package]] -name = "anes" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" - -[[package]] -name = "anstyle" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "038dfcf04a5feb68e9c60b21c9625a54c2c0616e79b72b0fd87075a056ae1d1b" - -[[package]] -name = "anyhow" -version = "1.0.86" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3d1d046238990b9cf5bcde22a3fb3584ee5cf65fb2765f454ed428c7a0063da" - -[[package]] -name = "arc-swap" -version = "1.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" - -[[package]] -name = "arrayref" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" - -[[package]] -name = "arrayvec" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" - -[[package]] -name = "arrow" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-arith", - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-csv", - "arrow-data", - "arrow-ipc", - "arrow-json", - "arrow-ord", - "arrow-row", - "arrow-schema", - "arrow-select", - "arrow-string", -] - -[[package]] -name = "arrow-arith" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "chrono", - "half", - "num", -] - -[[package]] -name = "arrow-array" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "chrono", - "chrono-tz 0.9.0", - "half", - "hashbrown", - "num", -] - -[[package]] -name = "arrow-buffer" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "bytes", - "half", - "num", -] - -[[package]] -name = "arrow-cast" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", - "atoi", - "base64", - "chrono", - "comfy-table", - "half", - "lexical-core", - "num", - "ryu", -] - -[[package]] -name = "arrow-csv" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", - "chrono", - "csv", - "csv-core", - "lazy_static", - "lexical-core", - "regex", -] - -[[package]] -name = "arrow-data" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-buffer", - "arrow-schema", - "half", - "num", -] - -[[package]] -name = "arrow-ipc" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", - "flatbuffers", - "lz4_flex", -] - -[[package]] -name = "arrow-json" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", - "chrono", - "half", - "indexmap", - "lexical-core", - "num", - "serde", - "serde_json", -] - -[[package]] -name = "arrow-ord" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", - "half", - "num", -] - -[[package]] -name = "arrow-row" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "half", - "hashbrown", -] - -[[package]] -name = "arrow-schema" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "bitflags 2.5.0", -] - -[[package]] -name = "arrow-select" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "num", -] - -[[package]] -name = "arrow-string" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", - "memchr", - "num", - "regex", - "regex-syntax", -] - -[[package]] -name = "assertables" -version = "7.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" - -[[package]] -name = "async-trait" -version = "0.1.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "atoi" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" -dependencies = [ - "num-traits", -] - -[[package]] -name = "autocfg" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" - -[[package]] -name = "backtrace" -version = "0.3.72" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17c6a35df3749d2e8bb1b7b21a976d82b15548788d2735b9d82f329268f71a11" -dependencies = [ - "addr2line", - "cc", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", -] - -[[package]] -name = "base64" -version = "0.22.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" - -[[package]] -name = "bitflags" -version = "1.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" - -[[package]] -name = "bitflags" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" - -[[package]] -name = "blake2" -version = "0.10.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" -dependencies = [ - "digest", -] - -[[package]] -name = "blake3" -version = "1.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30cca6d3674597c30ddf2c587bf8d9d65c9a84d2326d941cc79c9842dfe0ef52" -dependencies = [ - "arrayref", - "arrayvec", - "cc", - "cfg-if", - "constant_time_eq", -] - -[[package]] -name = "block-buffer" -version = "0.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" -dependencies = [ - "generic-array", -] - -[[package]] -name = "brotli" -version = "3.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", - "brotli-decompressor", -] - -[[package]] -name = "brotli-decompressor" -version = "2.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", -] - -[[package]] -name = "bumpalo" -version = "3.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" - -[[package]] -name = "bytemuck" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78834c15cb5d5efe3452d58b1e8ba890dd62d21907f867f383358198e56ebca5" - -[[package]] -name = "byteorder" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" - -[[package]] -name = "bytes" -version = "1.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" - -[[package]] -name = "cast" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" - -[[package]] -name = "cc" -version = "1.0.99" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695" -dependencies = [ - "jobserver", - "libc", - "once_cell", -] - -[[package]] -name = "cesu8" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "chrono" -version = "0.4.38" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" -dependencies = [ - "android-tzdata", - "iana-time-zone", - "num-traits", - "windows-targets 0.52.5", -] - -[[package]] -name = "chrono-tz" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d59ae0466b83e838b81a54256c39d5d7c20b9d7daa10510a242d9b75abd5936e" -dependencies = [ - "chrono", - "chrono-tz-build 0.2.1", - "phf", -] - -[[package]] -name = "chrono-tz" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" -dependencies = [ - "chrono", - "chrono-tz-build 0.3.0", - "phf", -] - -[[package]] -name = "chrono-tz-build" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" -dependencies = [ - "parse-zoneinfo", - "phf", - "phf_codegen", -] - -[[package]] -name = "chrono-tz-build" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" -dependencies = [ - "parse-zoneinfo", - "phf", - "phf_codegen", -] - -[[package]] -name = "ciborium" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" -dependencies = [ - "ciborium-io", - "ciborium-ll", - "serde", -] - -[[package]] -name = "ciborium-io" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" - -[[package]] -name = "ciborium-ll" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" -dependencies = [ - "ciborium-io", - "half", -] - -[[package]] -name = "clap" -version = "4.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9689a29b593160de5bc4aacab7b5d54fb52231de70122626c178e6a368994c7" -dependencies = [ - "clap_builder", -] - -[[package]] -name = "clap_builder" -version = "4.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e5387378c84f6faa26890ebf9f0a92989f8873d4d380467bcd0d8d8620424df" -dependencies = [ - "anstyle", - "clap_lex", -] - -[[package]] -name = "clap_lex" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" - -[[package]] -name = "combine" -version = "4.6.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" -dependencies = [ - "bytes", - "memchr", -] - -[[package]] -name = "comfy-table" -version = "7.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" -dependencies = [ - "strum", - "strum_macros", - "unicode-width", -] - -[[package]] -name = "const-random" -version = "0.1.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" -dependencies = [ - "const-random-macro", -] - -[[package]] -name = "const-random-macro" -version = "0.1.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" -dependencies = [ - "getrandom", - "once_cell", - "tiny-keccak", -] - -[[package]] -name = "constant_time_eq" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" - -[[package]] -name = "core-foundation-sys" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" - -[[package]] -name = "cpp_demangle" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8227005286ec39567949b33df9896bcadfa6051bccca2488129f108ca23119" -dependencies = [ - "cfg-if", -] - -[[package]] -name = "cpufeatures" -version = "0.2.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" -dependencies = [ - "libc", -] - -[[package]] -name = "crc32fast" -version = "1.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" -dependencies = [ - "cfg-if", -] - -[[package]] -name = "criterion" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" -dependencies = [ - "anes", - "cast", - "ciborium", - "clap", - "criterion-plot", - "is-terminal", - "itertools 0.10.5", - "num-traits", - "once_cell", - "oorandom", - "plotters", - "rayon", - "regex", - "serde", - "serde_derive", - "serde_json", - "tinytemplate", - "walkdir", -] - -[[package]] -name = "criterion-plot" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" -dependencies = [ - "cast", - "itertools 0.10.5", -] - -[[package]] -name = "crossbeam-deque" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" -dependencies = [ - "crossbeam-epoch", - "crossbeam-utils", -] - -[[package]] -name = "crossbeam-epoch" -version = "0.9.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" -dependencies = [ - "crossbeam-utils", -] - -[[package]] -name = "crossbeam-utils" -version = "0.8.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" - -[[package]] -name = "crunchy" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" - -[[package]] -name = "crypto-common" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" -dependencies = [ - "generic-array", - "typenum", -] - -[[package]] -name = "csv" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" -dependencies = [ - "csv-core", - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "csv-core" -version = "0.1.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" -dependencies = [ - "memchr", -] - -[[package]] -name = "dashmap" -version = "5.5.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" -dependencies = [ - "cfg-if", - "hashbrown", - "lock_api", - "once_cell", - "parking_lot_core", -] - -[[package]] -name = "datafusion" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-ipc", - "arrow-schema", - "async-trait", - "bytes", - "chrono", - "dashmap", - "datafusion-common", - "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions", - "datafusion-functions-aggregate", - "datafusion-optimizer", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "datafusion-physical-plan", - "datafusion-sql", - "futures", - "glob", - "half", - "hashbrown", - "indexmap", - "itertools 0.12.1", - "log", - "num_cpus", - "object_store", - "parking_lot", - "paste", - "pin-project-lite", - "rand", - "sqlparser", - "tempfile", - "tokio", - "url", - "uuid", -] - -[[package]] -name = "datafusion-comet" -version = "0.1.0" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-string", - "assertables", - "async-trait", - "brotli", - "byteorder", - "bytes", - "chrono", - "chrono-tz 0.8.6", - "crc32fast", - "criterion", - "datafusion", - "datafusion-common", - "datafusion-expr", - "datafusion-functions", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "flate2", - "futures", - "half", - "hashbrown", - "hex", - "itertools 0.11.0", - "jni", - "lazy_static", - "log", - "log4rs", - "lz4", - "mimalloc", - "num", - "once_cell", - "parking_lot", - "parquet", - "parquet-format", - "paste", - "pprof", - "prost 0.12.6", - "prost-build", - "rand", - "regex", - "serde", - "simd-adler32", - "snap", - "tempfile", - "thiserror", - "thrift 0.17.0", - "tokio", - "tokio-stream", - "twox-hash", - "unicode-segmentation", - "zstd", -] - -[[package]] -name = "datafusion-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-schema", - "chrono", - "half", - "hashbrown", - "instant", - "libc", - "num_cpus", - "object_store", - "sqlparser", -] - -[[package]] -name = "datafusion-common-runtime" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "tokio", -] - -[[package]] -name = "datafusion-execution" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "chrono", - "dashmap", - "datafusion-common", - "datafusion-expr", - "futures", - "hashbrown", - "log", - "object_store", - "parking_lot", - "rand", - "tempfile", - "url", -] - -[[package]] -name = "datafusion-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "chrono", - "datafusion-common", - "paste", - "serde_json", - "sqlparser", - "strum", - "strum_macros", -] - -[[package]] -name = "datafusion-functions" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "base64", - "blake2", - "blake3", - "chrono", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "hashbrown", - "hex", - "itertools 0.12.1", - "log", - "md-5", - "rand", - "regex", - "sha2", - "unicode-segmentation", - "uuid", -] - -[[package]] -name = "datafusion-functions-aggregate" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-schema", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr-common", - "log", - "paste", - "sqlparser", -] - -[[package]] -name = "datafusion-optimizer" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "async-trait", - "chrono", - "datafusion-common", - "datafusion-expr", - "datafusion-physical-expr", - "hashbrown", - "indexmap", - "itertools 0.12.1", - "log", - "regex-syntax", -] - -[[package]] -name = "datafusion-physical-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-ord", - "arrow-schema", - "arrow-string", - "base64", - "chrono", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate", - "datafusion-physical-expr-common", - "half", - "hashbrown", - "hex", - "indexmap", - "itertools 0.12.1", - "log", - "paste", - "petgraph", - "regex", -] - -[[package]] -name = "datafusion-physical-expr-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "datafusion-common", - "datafusion-expr", - "rand", -] - -[[package]] -name = "datafusion-physical-plan" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-ord", - "arrow-schema", - "async-trait", - "chrono", - "datafusion-common", - "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "futures", - "half", - "hashbrown", - "indexmap", - "itertools 0.12.1", - "log", - "once_cell", - "parking_lot", - "pin-project-lite", - "rand", - "tokio", -] - -[[package]] -name = "datafusion-sql" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "arrow-array", - "arrow-schema", - "datafusion-common", - "datafusion-expr", - "log", - "regex", - "sqlparser", - "strum", -] - -[[package]] -name = "debugid" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" -dependencies = [ - "uuid", -] - -[[package]] -name = "derivative" -version = "2.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" -dependencies = [ - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "destructure_traitobject" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c877555693c14d2f84191cfd3ad8582790fc52b5e2274b40b59cf5f5cea25c7" - -[[package]] -name = "digest" -version = "0.10.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" -dependencies = [ - "block-buffer", - "crypto-common", - "subtle", -] - -[[package]] -name = "doc-comment" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" - -[[package]] -name = "either" -version = "1.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dca9240753cf90908d7e4aac30f630662b02aebaa1b58a3cadabdb23385b58b" - -[[package]] -name = "equivalent" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" - -[[package]] -name = "errno" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" -dependencies = [ - "libc", - "windows-sys 0.52.0", -] - -[[package]] -name = "fastrand" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" - -[[package]] -name = "findshlibs" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" -dependencies = [ - "cc", - "lazy_static", - "libc", - "winapi", -] - -[[package]] -name = "fixedbitset" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" - -[[package]] -name = "flatbuffers" -version = "24.3.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" -dependencies = [ - "bitflags 1.3.2", - "rustc_version", -] - -[[package]] -name = "flate2" -version = "1.0.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" -dependencies = [ - "crc32fast", - "miniz_oxide", -] - -[[package]] -name = "fnv" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" - -[[package]] -name = "form_urlencoded" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" -dependencies = [ - "percent-encoding", -] - -[[package]] -name = "futures" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" -dependencies = [ - "futures-channel", - "futures-core", - "futures-executor", - "futures-io", - "futures-sink", - "futures-task", - "futures-util", -] - -[[package]] -name = "futures-channel" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" -dependencies = [ - "futures-core", - "futures-sink", -] - -[[package]] -name = "futures-core" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" - -[[package]] -name = "futures-executor" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" -dependencies = [ - "futures-core", - "futures-task", - "futures-util", -] - -[[package]] -name = "futures-io" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" - -[[package]] -name = "futures-macro" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "futures-sink" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" - -[[package]] -name = "futures-task" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" - -[[package]] -name = "futures-util" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" -dependencies = [ - "futures-channel", - "futures-core", - "futures-io", - "futures-macro", - "futures-sink", - "futures-task", - "memchr", - "pin-project-lite", - "pin-utils", - "slab", -] - -[[package]] -name = "generic-array" -version = "0.14.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" -dependencies = [ - "typenum", - "version_check", -] - -[[package]] -name = "getrandom" -version = "0.2.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" -dependencies = [ - "cfg-if", - "libc", - "wasi", -] - -[[package]] -name = "gimli" -version = "0.29.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" - -[[package]] -name = "glob" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" - -[[package]] -name = "half" -version = "2.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dd08c532ae367adf81c312a4580bc67f1d0fe8bc9c460520283f4c0ff277888" -dependencies = [ - "cfg-if", - "crunchy", - "num-traits", -] - -[[package]] -name = "hashbrown" -version = "0.14.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" -dependencies = [ - "ahash", - "allocator-api2", -] - -[[package]] -name = "heck" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" -dependencies = [ - "unicode-segmentation", -] - -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - -[[package]] -name = "heck" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" - -[[package]] -name = "hermit-abi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" - -[[package]] -name = "hex" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" - -[[package]] -name = "home" -version = "0.5.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" -dependencies = [ - "windows-sys 0.52.0", -] - -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" - -[[package]] -name = "iana-time-zone" -version = "0.1.60" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" -dependencies = [ - "android_system_properties", - "core-foundation-sys", - "iana-time-zone-haiku", - "js-sys", - "wasm-bindgen", - "windows-core", -] - -[[package]] -name = "iana-time-zone-haiku" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" -dependencies = [ - "cc", -] - -[[package]] -name = "idna" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" -dependencies = [ - "unicode-bidi", - "unicode-normalization", -] - -[[package]] -name = "indexmap" -version = "2.2.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" -dependencies = [ - "equivalent", - "hashbrown", -] - -[[package]] -name = "inferno" -version = "0.11.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" -dependencies = [ - "ahash", - "indexmap", - "is-terminal", - "itoa", - "log", - "num-format", - "once_cell", - "quick-xml", - "rgb", - "str_stack", -] - -[[package]] -name = "instant" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" -dependencies = [ - "cfg-if", - "js-sys", - "wasm-bindgen", - "web-sys", -] - -[[package]] -name = "integer-encoding" -version = "1.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" - -[[package]] -name = "integer-encoding" -version = "3.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" - -[[package]] -name = "is-terminal" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" -dependencies = [ - "hermit-abi", - "libc", - "windows-sys 0.52.0", -] - -[[package]] -name = "itertools" -version = "0.10.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" -dependencies = [ - "either", -] - -[[package]] -name = "itertools" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" -dependencies = [ - "either", -] - -[[package]] -name = "itertools" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" -dependencies = [ - "either", -] - -[[package]] -name = "itoa" -version = "1.0.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" - -[[package]] -name = "java-locator" -version = "0.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2abecabd9961c5e01405a6426687fcf1bd94a269927137e4c3cc1a7419b93fd" -dependencies = [ - "glob", - "lazy_static", -] - -[[package]] -name = "jni" -version = "0.21.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a87aa2bb7d2af34197c04845522473242e1aa17c12f4935d5856491a7fb8c97" -dependencies = [ - "cesu8", - "cfg-if", - "combine", - "java-locator", - "jni-sys", - "libloading", - "log", - "thiserror", - "walkdir", - "windows-sys 0.45.0", -] - -[[package]] -name = "jni-sys" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" - -[[package]] -name = "jobserver" -version = "0.1.31" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" -dependencies = [ - "libc", -] - -[[package]] -name = "js-sys" -version = "0.3.69" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" -dependencies = [ - "wasm-bindgen", -] - -[[package]] -name = "lazy_static" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" - -[[package]] -name = "lexical-core" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" -dependencies = [ - "lexical-parse-float", - "lexical-parse-integer", - "lexical-util", - "lexical-write-float", - "lexical-write-integer", -] - -[[package]] -name = "lexical-parse-float" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" -dependencies = [ - "lexical-parse-integer", - "lexical-util", - "static_assertions", -] - -[[package]] -name = "lexical-parse-integer" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" -dependencies = [ - "lexical-util", - "static_assertions", -] - -[[package]] -name = "lexical-util" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" -dependencies = [ - "static_assertions", -] - -[[package]] -name = "lexical-write-float" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" -dependencies = [ - "lexical-util", - "lexical-write-integer", - "static_assertions", -] - -[[package]] -name = "lexical-write-integer" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" -dependencies = [ - "lexical-util", - "static_assertions", -] - -[[package]] -name = "libc" -version = "0.2.155" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" - -[[package]] -name = "libloading" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b67380fd3b2fbe7527a606e18729d21c6f3951633d0500574c4dc22d2d638b9f" -dependencies = [ - "cfg-if", - "winapi", -] - -[[package]] -name = "libm" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" - -[[package]] -name = "libmimalloc-sys" -version = "0.1.38" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e7bb23d733dfcc8af652a78b7bf232f0e967710d044732185e561e47c0336b6" -dependencies = [ - "cc", - "libc", -] - -[[package]] -name = "linux-raw-sys" -version = "0.4.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" - -[[package]] -name = "lock_api" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" -dependencies = [ - "autocfg", - "scopeguard", -] - -[[package]] -name = "log" -version = "0.4.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" -dependencies = [ - "serde", -] - -[[package]] -name = "log-mdc" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" - -[[package]] -name = "log4rs" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0816135ae15bd0391cf284eab37e6e3ee0a6ee63d2ceeb659862bd8d0a984ca6" -dependencies = [ - "anyhow", - "arc-swap", - "chrono", - "derivative", - "fnv", - "humantime", - "libc", - "log", - "log-mdc", - "once_cell", - "parking_lot", - "rand", - "serde", - "serde-value", - "serde_json", - "serde_yaml", - "thiserror", - "thread-id", - "typemap-ors", - "winapi", -] - -[[package]] -name = "lz4" -version = "1.24.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" -dependencies = [ - "libc", - "lz4-sys", -] - -[[package]] -name = "lz4-sys" -version = "1.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" -dependencies = [ - "cc", - "libc", -] - -[[package]] -name = "lz4_flex" -version = "0.11.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" -dependencies = [ - "twox-hash", -] - -[[package]] -name = "md-5" -version = "0.10.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" -dependencies = [ - "cfg-if", - "digest", -] - -[[package]] -name = "memchr" -version = "2.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" - -[[package]] -name = "memmap2" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe751422e4a8caa417e13c3ea66452215d7d63e19e604f4980461212f3ae1322" -dependencies = [ - "libc", -] - -[[package]] -name = "mimalloc" -version = "0.1.42" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9186d86b79b52f4a77af65604b51225e8db1d6ee7e3f41aec1e40829c71a176" -dependencies = [ - "libmimalloc-sys", -] - -[[package]] -name = "miniz_oxide" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87dfd01fe195c66b572b37921ad8803d010623c0aca821bea2302239d155cdae" -dependencies = [ - "adler", -] - -[[package]] -name = "multimap" -version = "0.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" - -[[package]] -name = "nix" -version = "0.26.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "598beaf3cc6fdd9a5dfb1630c2800c7acd31df7aaf0f565796fba2b53ca1af1b" -dependencies = [ - "bitflags 1.3.2", - "cfg-if", - "libc", -] - -[[package]] -name = "num" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" -dependencies = [ - "num-bigint", - "num-complex", - "num-integer", - "num-iter", - "num-rational", - "num-traits", -] - -[[package]] -name = "num-bigint" -version = "0.4.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7" -dependencies = [ - "num-integer", - "num-traits", -] - -[[package]] -name = "num-complex" -version = "0.4.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" -dependencies = [ - "num-traits", -] - -[[package]] -name = "num-format" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" -dependencies = [ - "arrayvec", - "itoa", -] - -[[package]] -name = "num-integer" -version = "0.1.46" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" -dependencies = [ - "num-traits", -] - -[[package]] -name = "num-iter" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-rational" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" -dependencies = [ - "num-bigint", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-traits" -version = "0.2.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" -dependencies = [ - "autocfg", - "libm", -] - -[[package]] -name = "num_cpus" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" -dependencies = [ - "hermit-abi", - "libc", -] - -[[package]] -name = "object" -version = "0.35.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8ec7ab813848ba4522158d5517a6093db1ded27575b070f4177b8d12b41db5e" -dependencies = [ - "memchr", -] - -[[package]] -name = "object_store" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbebfd32c213ba1907fa7a9c9138015a8de2b43e30c5aa45b18f7deb46786ad6" -dependencies = [ - "async-trait", - "bytes", - "chrono", - "futures", - "humantime", - "itertools 0.12.1", - "parking_lot", - "percent-encoding", - "snafu", - "tokio", - "tracing", - "url", - "walkdir", -] - -[[package]] -name = "once_cell" -version = "1.19.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" - -[[package]] -name = "oorandom" -version = "11.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" - -[[package]] -name = "ordered-float" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] - -[[package]] -name = "ordered-float" -version = "2.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" -dependencies = [ - "num-traits", -] - -[[package]] -name = "parking_lot" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" -dependencies = [ - "lock_api", - "parking_lot_core", -] - -[[package]] -name = "parking_lot_core" -version = "0.9.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" -dependencies = [ - "cfg-if", - "libc", - "redox_syscall", - "smallvec", - "windows-targets 0.52.5", -] - -[[package]] -name = "parquet" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "bytes", - "chrono", - "half", - "hashbrown", - "num", - "num-bigint", - "paste", - "seq-macro", - "thrift 0.17.0", - "twox-hash", -] - -[[package]] -name = "parquet-format" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" -dependencies = [ - "thrift 0.13.0", -] - -[[package]] -name = "parse-zoneinfo" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f2a05b18d44e2957b88f96ba460715e295bc1d7510468a2f3d3b44535d26c24" -dependencies = [ - "regex", -] - -[[package]] -name = "paste" -version = "1.0.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" - -[[package]] -name = "percent-encoding" -version = "2.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" - -[[package]] -name = "petgraph" -version = "0.6.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" -dependencies = [ - "fixedbitset", - "indexmap", -] - -[[package]] -name = "phf" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" -dependencies = [ - "phf_shared", -] - -[[package]] -name = "phf_codegen" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8d39688d359e6b34654d328e262234662d16cc0f60ec8dcbe5e718709342a5a" -dependencies = [ - "phf_generator", - "phf_shared", -] - -[[package]] -name = "phf_generator" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" -dependencies = [ - "phf_shared", - "rand", -] - -[[package]] -name = "phf_shared" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" -dependencies = [ - "siphasher", -] - -[[package]] -name = "pin-project-lite" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" - -[[package]] -name = "pin-utils" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" - -[[package]] -name = "pkg-config" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" - -[[package]] -name = "plotters" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a15b6eccb8484002195a3e44fe65a4ce8e93a625797a063735536fd59cb01cf3" -dependencies = [ - "num-traits", - "plotters-backend", - "plotters-svg", - "wasm-bindgen", - "web-sys", -] - -[[package]] -name = "plotters-backend" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "414cec62c6634ae900ea1c56128dfe87cf63e7caece0852ec76aba307cebadb7" - -[[package]] -name = "plotters-svg" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81b30686a7d9c3e010b84284bdd26a29f2138574f52f5eb6f794fc0ad924e705" -dependencies = [ - "plotters-backend", -] - -[[package]] -name = "pprof" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" -dependencies = [ - "backtrace", - "cfg-if", - "findshlibs", - "inferno", - "libc", - "log", - "nix", - "once_cell", - "parking_lot", - "smallvec", - "symbolic-demangle", - "tempfile", - "thiserror", -] - -[[package]] -name = "ppv-lite86" -version = "0.2.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" - -[[package]] -name = "proc-macro2" -version = "1.0.85" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" -dependencies = [ - "unicode-ident", -] - -[[package]] -name = "prost" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" -dependencies = [ - "bytes", - "prost-derive 0.9.0", -] - -[[package]] -name = "prost" -version = "0.12.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" -dependencies = [ - "bytes", - "prost-derive 0.12.6", -] - -[[package]] -name = "prost-build" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" -dependencies = [ - "bytes", - "heck 0.3.3", - "itertools 0.10.5", - "lazy_static", - "log", - "multimap", - "petgraph", - "prost 0.9.0", - "prost-types", - "regex", - "tempfile", - "which", -] - -[[package]] -name = "prost-derive" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" -dependencies = [ - "anyhow", - "itertools 0.10.5", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "prost-derive" -version = "0.12.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" -dependencies = [ - "anyhow", - "itertools 0.12.1", - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "prost-types" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" -dependencies = [ - "bytes", - "prost 0.9.0", -] - -[[package]] -name = "quick-xml" -version = "0.26.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f50b1c63b38611e7d4d7f68b82d3ad0cc71a2ad2e7f61fc10f1328d917c93cd" -dependencies = [ - "memchr", -] - -[[package]] -name = "quote" -version = "1.0.36" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" -dependencies = [ - "proc-macro2", -] - -[[package]] -name = "rand" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" -dependencies = [ - "libc", - "rand_chacha", - "rand_core", -] - -[[package]] -name = "rand_chacha" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" -dependencies = [ - "ppv-lite86", - "rand_core", -] - -[[package]] -name = "rand_core" -version = "0.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" -dependencies = [ - "getrandom", -] - -[[package]] -name = "rayon" -version = "1.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" -dependencies = [ - "either", - "rayon-core", -] - -[[package]] -name = "rayon-core" -version = "1.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" -dependencies = [ - "crossbeam-deque", - "crossbeam-utils", -] - -[[package]] -name = "redox_syscall" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e" -dependencies = [ - "bitflags 2.5.0", -] - -[[package]] -name = "regex" -version = "1.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c117dbdfde9c8308975b6a18d71f3f385c89461f7b3fb054288ecf2a2058ba4c" -dependencies = [ - "aho-corasick", - "memchr", - "regex-automata", - "regex-syntax", -] - -[[package]] -name = "regex-automata" -version = "0.4.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" -dependencies = [ - "aho-corasick", - "memchr", - "regex-syntax", -] - -[[package]] -name = "regex-syntax" -version = "0.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adad44e29e4c806119491a7f06f03de4d1af22c3a680dd47f1e6e179439d1f56" - -[[package]] -name = "rgb" -version = "0.8.37" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05aaa8004b64fd573fc9d002f4e632d51ad4f026c2b5ba95fcb6c2f32c2c47d8" -dependencies = [ - "bytemuck", -] - -[[package]] -name = "rustc-demangle" -version = "0.1.24" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" - -[[package]] -name = "rustc_version" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" -dependencies = [ - "semver", -] - -[[package]] -name = "rustix" -version = "0.38.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" -dependencies = [ - "bitflags 2.5.0", - "errno", - "libc", - "linux-raw-sys", - "windows-sys 0.52.0", -] - -[[package]] -name = "rustversion" -version = "1.0.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "955d28af4278de8121b7ebeb796b6a45735dc01436d898801014aced2773a3d6" - -[[package]] -name = "ryu" -version = "1.0.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" - -[[package]] -name = "same-file" -version = "1.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" -dependencies = [ - "winapi-util", -] - -[[package]] -name = "scopeguard" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" - -[[package]] -name = "semver" -version = "1.0.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" - -[[package]] -name = "seq-macro" -version = "0.3.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" - -[[package]] -name = "serde" -version = "1.0.203" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" -dependencies = [ - "serde_derive", -] - -[[package]] -name = "serde-value" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" -dependencies = [ - "ordered-float 2.10.1", - "serde", -] - -[[package]] -name = "serde_derive" -version = "1.0.203" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "serde_json" -version = "1.0.117" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "455182ea6142b14f93f4bc5320a2b31c1f266b66a4a5c858b013302a5d8cbfc3" -dependencies = [ - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "serde_yaml" -version = "0.9.34+deprecated" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" -dependencies = [ - "indexmap", - "itoa", - "ryu", - "serde", - "unsafe-libyaml", -] - -[[package]] -name = "sha2" -version = "0.10.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" -dependencies = [ - "cfg-if", - "cpufeatures", - "digest", -] - -[[package]] -name = "simd-adler32" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" - -[[package]] -name = "siphasher" -version = "0.3.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" - -[[package]] -name = "slab" -version = "0.4.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" -dependencies = [ - "autocfg", -] - -[[package]] -name = "smallvec" -version = "1.13.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" - -[[package]] -name = "snafu" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" -dependencies = [ - "doc-comment", - "snafu-derive", -] - -[[package]] -name = "snafu-derive" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" -dependencies = [ - "heck 0.4.1", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "snap" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" - -[[package]] -name = "sqlparser" -version = "0.47.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "295e9930cd7a97e58ca2a070541a3ca502b17f5d1fa7157376d0fabd85324f25" -dependencies = [ - "log", - "sqlparser_derive", -] - -[[package]] -name = "sqlparser_derive" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "stable_deref_trait" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" - -[[package]] -name = "static_assertions" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" - -[[package]] -name = "str_stack" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" - -[[package]] -name = "strum" -version = "0.26.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" -dependencies = [ - "strum_macros", -] - -[[package]] -name = "strum_macros" -version = "0.26.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" -dependencies = [ - "heck 0.5.0", - "proc-macro2", - "quote", - "rustversion", - "syn 2.0.66", -] - -[[package]] -name = "subtle" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" - -[[package]] -name = "symbolic-common" -version = "12.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71297dc3e250f7dbdf8adb99e235da783d690f5819fdeb4cce39d9cfb0aca9f1" -dependencies = [ - "debugid", - "memmap2", - "stable_deref_trait", - "uuid", -] - -[[package]] -name = "symbolic-demangle" -version = "12.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "424fa2c9bf2c862891b9cfd354a752751a6730fd838a4691e7f6c2c7957b9daf" -dependencies = [ - "cpp_demangle", - "rustc-demangle", - "symbolic-common", -] - -[[package]] -name = "syn" -version = "1.0.109" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "syn" -version = "2.0.66" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "tempfile" -version = "3.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" -dependencies = [ - "cfg-if", - "fastrand", - "rustix", - "windows-sys 0.52.0", -] - -[[package]] -name = "thiserror" -version = "1.0.61" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" -dependencies = [ - "thiserror-impl", -] - -[[package]] -name = "thiserror-impl" -version = "1.0.61" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "thread-id" -version = "4.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0ec81c46e9eb50deaa257be2f148adf052d1fb7701cfd55ccfab2525280b70b" -dependencies = [ - "libc", - "winapi", -] - -[[package]] -name = "threadpool" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" -dependencies = [ - "num_cpus", -] - -[[package]] -name = "thrift" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" -dependencies = [ - "byteorder", - "integer-encoding 1.1.7", - "log", - "ordered-float 1.1.1", - "threadpool", -] - -[[package]] -name = "thrift" -version = "0.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" -dependencies = [ - "byteorder", - "integer-encoding 3.0.4", - "log", - "ordered-float 2.10.1", - "threadpool", -] - -[[package]] -name = "tiny-keccak" -version = "2.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" -dependencies = [ - "crunchy", -] - -[[package]] -name = "tinytemplate" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" -dependencies = [ - "serde", - "serde_json", -] - -[[package]] -name = "tinyvec" -version = "1.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" -dependencies = [ - "tinyvec_macros", -] - -[[package]] -name = "tinyvec_macros" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" - -[[package]] -name = "tokio" -version = "1.38.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" -dependencies = [ - "backtrace", - "bytes", - "num_cpus", - "pin-project-lite", - "tokio-macros", -] - -[[package]] -name = "tokio-macros" -version = "2.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - -[[package]] -name = "tracing" -version = "0.1.40" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" -dependencies = [ - "pin-project-lite", - "tracing-attributes", - "tracing-core", -] - -[[package]] -name = "tracing-attributes" -version = "0.1.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "tracing-core" -version = "0.1.32" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" -dependencies = [ - "once_cell", -] - -[[package]] -name = "twox-hash" -version = "1.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" -dependencies = [ - "cfg-if", - "rand", - "static_assertions", -] - -[[package]] -name = "typemap-ors" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a68c24b707f02dd18f1e4ccceb9d49f2058c2fb86384ef9972592904d7a28867" -dependencies = [ - "unsafe-any-ors", -] - -[[package]] -name = "typenum" -version = "1.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" - -[[package]] -name = "unicode-bidi" -version = "0.3.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" - -[[package]] -name = "unicode-ident" -version = "1.0.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" - -[[package]] -name = "unicode-normalization" -version = "0.1.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" -dependencies = [ - "tinyvec", -] - -[[package]] -name = "unicode-segmentation" -version = "1.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" - -[[package]] -name = "unicode-width" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" - -[[package]] -name = "unsafe-any-ors" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0a303d30665362d9680d7d91d78b23f5f899504d4f08b3c4cf08d055d87c0ad" -dependencies = [ - "destructure_traitobject", -] - -[[package]] -name = "unsafe-libyaml" -version = "0.2.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" - -[[package]] -name = "url" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" -dependencies = [ - "form_urlencoded", - "idna", - "percent-encoding", -] - -[[package]] -name = "uuid" -version = "1.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0" -dependencies = [ - "getrandom", -] - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" - -[[package]] -name = "walkdir" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" -dependencies = [ - "same-file", - "winapi-util", -] - -[[package]] -name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" - -[[package]] -name = "wasm-bindgen" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" -dependencies = [ - "cfg-if", - "wasm-bindgen-macro", -] - -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" -dependencies = [ - "bumpalo", - "log", - "once_cell", - "proc-macro2", - "quote", - "syn 2.0.66", - "wasm-bindgen-shared", -] - -[[package]] -name = "wasm-bindgen-macro" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" -dependencies = [ - "quote", - "wasm-bindgen-macro-support", -] - -[[package]] -name = "wasm-bindgen-macro-support" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", - "wasm-bindgen-backend", - "wasm-bindgen-shared", -] - -[[package]] -name = "wasm-bindgen-shared" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" - -[[package]] -name = "web-sys" -version = "0.3.69" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" -dependencies = [ - "js-sys", - "wasm-bindgen", -] - -[[package]] -name = "which" -version = "4.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" -dependencies = [ - "either", - "home", - "once_cell", - "rustix", -] - -[[package]] -name = "winapi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" -dependencies = [ - "winapi-i686-pc-windows-gnu", - "winapi-x86_64-pc-windows-gnu", -] - -[[package]] -name = "winapi-i686-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" - -[[package]] -name = "winapi-util" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" -dependencies = [ - "windows-sys 0.52.0", -] - -[[package]] -name = "winapi-x86_64-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" - -[[package]] -name = "windows-core" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" -dependencies = [ - "windows-targets 0.52.5", -] - -[[package]] -name = "windows-sys" -version = "0.45.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" -dependencies = [ - "windows-targets 0.42.2", -] - -[[package]] -name = "windows-sys" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" -dependencies = [ - "windows-targets 0.52.5", -] - -[[package]] -name = "windows-targets" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" -dependencies = [ - "windows_aarch64_gnullvm 0.42.2", - "windows_aarch64_msvc 0.42.2", - "windows_i686_gnu 0.42.2", - "windows_i686_msvc 0.42.2", - "windows_x86_64_gnu 0.42.2", - "windows_x86_64_gnullvm 0.42.2", - "windows_x86_64_msvc 0.42.2", -] - -[[package]] -name = "windows-targets" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" -dependencies = [ - "windows_aarch64_gnullvm 0.52.5", - "windows_aarch64_msvc 0.52.5", - "windows_i686_gnu 0.52.5", - "windows_i686_gnullvm", - "windows_i686_msvc 0.52.5", - "windows_x86_64_gnu 0.52.5", - "windows_x86_64_gnullvm 0.52.5", - "windows_x86_64_msvc 0.52.5", -] - -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" - -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" - -[[package]] -name = "windows_aarch64_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" - -[[package]] -name = "windows_aarch64_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" - -[[package]] -name = "windows_i686_gnu" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" - -[[package]] -name = "windows_i686_gnu" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" - -[[package]] -name = "windows_i686_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" - -[[package]] -name = "windows_i686_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" - -[[package]] -name = "windows_i686_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" - -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" - -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" - -[[package]] -name = "windows_x86_64_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" - -[[package]] -name = "windows_x86_64_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" - -[[package]] -name = "zerocopy" -version = "0.7.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae87e3fcd617500e5d106f0380cf7b77f3c6092aae37191433159dda23cfb087" -dependencies = [ - "zerocopy-derive", -] - -[[package]] -name = "zerocopy-derive" -version = "0.7.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "zstd" -version = "0.11.2+zstd.1.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" -dependencies = [ - "zstd-safe", -] - -[[package]] -name = "zstd-safe" -version = "5.0.2+zstd.1.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" -dependencies = [ - "libc", - "zstd-sys", -] - -[[package]] -name = "zstd-sys" -version = "2.0.10+zstd.1.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" -dependencies = [ - "cc", - "pkg-config", -] diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 160db2949..3820bdd38 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -31,13 +31,16 @@ include = [ "Cargo.toml", ] +# this crate is used in the Spark plugin and does not contain public Rust APIs so we do not publish this crate +publish = false + [dependencies] -arrow = { version = "52.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "52.1.0" } -arrow-buffer = { version = "52.1.0" } -arrow-data = { version = "52.1.0" } -arrow-schema = { version = "52.1.0" } -parquet = { version = "52.1.0", default-features = false, features = ["experimental"] } +arrow = { workspace = true } +arrow-array = { workspace = true } +arrow-buffer = { workspace = true } +arrow-data = { workspace = true } +arrow-schema = { workspace = true } +parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } @@ -64,17 +67,17 @@ itertools = "0.11.0" chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } paste = "1.0.14" -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1" } -datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["crypto_expressions"] } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } -datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-common = { workspace = true } +datafusion = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-expr = { workspace = true } unicode-segmentation = "^1.10.1" once_cell = "1.18.0" regex = "1.9.6" crc32fast = "1.3.2" simd-adler32 = "0.3.7" +datafusion-comet-spark-expr = { path = "../spark-expr", version = "0.1.0" } [build-dependencies] prost-build = "0.9.0" @@ -91,13 +94,6 @@ twox-hash = "1.6.3" [features] default = [] -[profile.release] -debug = true -overflow-checks = false -lto = "thin" -codegen-units = 1 -strip = "debuginfo" - [lib] name = "comet" # "rlib" is for benchmarking with criterion. diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 385857f1e..98b422dce 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -24,10 +24,8 @@ pub mod if_expr; mod normalize_nan; pub mod scalar_funcs; pub use normalize_nan::NormalizeNaNAndZero; -use prost::DecodeError; -use crate::{errors::CometError, execution::spark_expression}; -pub mod abs; +use crate::errors::CometError; pub mod avg; pub mod avg_decimal; pub mod bloom_filter_might_contain; @@ -46,24 +44,7 @@ mod utils; pub mod variance; pub mod xxhash64; -#[derive(Debug, Hash, PartialEq, Clone, Copy)] -pub enum EvalMode { - Legacy, - Ansi, - Try, -} - -impl TryFrom for EvalMode { - type Error = DecodeError; - - fn try_from(value: i32) -> Result { - match spark_expression::EvalMode::try_from(value)? { - spark_expression::EvalMode::Legacy => Ok(EvalMode::Legacy), - spark_expression::EvalMode::Try => Ok(EvalMode::Try), - spark_expression::EvalMode::Ansi => Ok(EvalMode::Ansi), - } - } -} +pub use datafusion_comet_spark_expr::EvalMode; fn arithmetic_overflow_error(from_type: &str) -> CometError { CometError::ArithmeticOverflow { diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 360380400..ee208ac74 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -107,7 +107,8 @@ use crate::{ }, }; -use super::expressions::{abs::CometAbsFunc, create_named_struct::CreateNamedStruct, EvalMode}; +use super::expressions::{create_named_struct::CreateNamedStruct, EvalMode}; +use datafusion_comet_spark_expr::abs::Abs; // For clippy error on type_complexity. type ExecResult = Result; @@ -368,7 +369,7 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let timezone = expr.timezone.clone(); - let eval_mode = expr.eval_mode.try_into()?; + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; Ok(Arc::new(Cast::new(child, datatype, eval_mode, timezone))) } @@ -507,8 +508,8 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; let return_type = child.data_type(&input_schema)?; let args = vec![child]; - let eval_mode = expr.eval_mode.try_into()?; - let comet_abs = Arc::new(ScalarUDF::new_from_impl(CometAbsFunc::new( + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; + let comet_abs = Arc::new(ScalarUDF::new_from_impl(Abs::new( eval_mode, return_type.to_string(), )?)); @@ -1814,6 +1815,14 @@ fn rewrite_physical_expr( Ok(expr.rewrite(&mut rewriter).data()?) } +fn from_protobuf_eval_mode(value: i32) -> Result { + match spark_expression::EvalMode::try_from(value)? { + spark_expression::EvalMode::Legacy => Ok(EvalMode::Legacy), + spark_expression::EvalMode::Try => Ok(EvalMode::Try), + spark_expression::EvalMode::Ansi => Ok(EvalMode::Ansi), + } +} + #[cfg(test)] mod tests { use std::{sync::Arc, task::Poll}; diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml new file mode 100644 index 000000000..d10d04944 --- /dev/null +++ b/native/spark-expr/Cargo.toml @@ -0,0 +1,38 @@ +# 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. + +[package] +name = "datafusion-comet-spark-expr" +description = "DataFusion expressions that emulate Apache Spark's behavior" +version = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +authors = { workspace = true } +readme = { workspace = true } +license = { workspace = true } +edition = { workspace = true } + +[dependencies] +arrow = { workspace = true } +arrow-schema = { workspace = true } +datafusion = { workspace = true } +datafusion-common = { workspace = true } +datafusion-functions = { workspace = true } + +[lib] +name = "datafusion_comet_spark_expr" +path = "src/lib.rs" diff --git a/native/spark-expr/README.md b/native/spark-expr/README.md new file mode 100644 index 000000000..a7ee75363 --- /dev/null +++ b/native/spark-expr/README.md @@ -0,0 +1,23 @@ + + +# datafusion-comet-spark-expr: Spark-compatible Expressions + +This crate provides Apache Spark-compatible expressions for use with DataFusion and is maintained as part of the +[Apache DataFusion Comet](https://github.com/apache/datafusion-comet/) subproject. \ No newline at end of file diff --git a/native/core/src/execution/datafusion/expressions/abs.rs b/native/spark-expr/src/abs.rs similarity index 81% rename from native/core/src/execution/datafusion/expressions/abs.rs rename to native/spark-expr/src/abs.rs index a037e5cbc..198a96e57 100644 --- a/native/core/src/execution/datafusion/expressions/abs.rs +++ b/native/spark-expr/src/abs.rs @@ -15,26 +15,29 @@ // specific language governing permissions and limitations // under the License. +//! Spark-compatible implementation of abs function + +use std::{any::Any, sync::Arc}; + use arrow::datatypes::DataType; use arrow_schema::ArrowError; + use datafusion::logical_expr::{ColumnarValue, ScalarUDFImpl, Signature}; use datafusion_common::DataFusionError; use datafusion_functions::math; -use std::{any::Any, sync::Arc}; - -use crate::execution::operators::ExecutionError; -use super::{arithmetic_overflow_error, EvalMode}; +use super::{EvalMode, SparkError}; +/// Spark-compatible ABS expression #[derive(Debug)] -pub struct CometAbsFunc { +pub struct Abs { inner_abs_func: Arc, eval_mode: EvalMode, data_type_name: String, } -impl CometAbsFunc { - pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result { +impl Abs { + pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result { if let EvalMode::Legacy | EvalMode::Ansi = eval_mode { Ok(Self { inner_abs_func: math::abs().inner().clone(), @@ -42,7 +45,7 @@ impl CometAbsFunc { data_type_name, }) } else { - Err(ExecutionError::GeneralError(format!( + Err(DataFusionError::Execution(format!( "Invalid EvalMode: \"{:?}\"", eval_mode ))) @@ -50,7 +53,7 @@ impl CometAbsFunc { } } -impl ScalarUDFImpl for CometAbsFunc { +impl ScalarUDFImpl for Abs { fn as_any(&self) -> &dyn Any { self } @@ -68,17 +71,15 @@ impl ScalarUDFImpl for CometAbsFunc { fn invoke(&self, args: &[ColumnarValue]) -> Result { match self.inner_abs_func.invoke(args) { - Err(DataFusionError::ArrowError(ArrowError::ComputeError(msg), trace)) + Err(DataFusionError::ArrowError(ArrowError::ComputeError(msg), _)) if msg.contains("overflow") => { if self.eval_mode == EvalMode::Legacy { Ok(args[0].clone()) } else { - let msg = arithmetic_overflow_error(&self.data_type_name).to_string(); - Err(DataFusionError::ArrowError( - ArrowError::ComputeError(msg), - trace, - )) + Err(DataFusionError::External(Box::new( + SparkError::ArithmeticOverflow(self.data_type_name.clone()), + ))) } } other => other, diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs new file mode 100644 index 000000000..3873754be --- /dev/null +++ b/native/spark-expr/src/lib.rs @@ -0,0 +1,56 @@ +// 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::error::Error; +use std::fmt::{Display, Formatter}; + +pub mod abs; + +/// Spark supports three evaluation modes when evaluating expressions, which affect +/// the behavior when processing input values that are invalid or would result in an +/// error, such as divide by zero errors, and also affects behavior when converting +/// between types. +#[derive(Debug, Hash, PartialEq, Clone, Copy)] +pub enum EvalMode { + /// Legacy is the default behavior in Spark prior to Spark 4.0. This mode silently ignores + /// or replaces errors during SQL operations. Operations resulting in errors (like + /// division by zero) will produce NULL values instead of failing. Legacy mode also + /// enables implicit type conversions. + Legacy, + /// Adheres to the ANSI SQL standard for error handling by throwing exceptions for + /// operations that result in errors. Does not perform implicit type conversions. + Ansi, + /// Same as Ansi mode, except that it converts errors to NULL values without + /// failing the entire query. + Try, +} + +#[derive(Debug)] +pub enum SparkError { + ArithmeticOverflow(String), +} + +impl Error for SparkError {} + +impl Display for SparkError { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::ArithmeticOverflow(data_type) => + write!(f, "[ARITHMETIC_OVERFLOW] {} overflow. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.", data_type) + } + } +} From 1dc092f67a0714fe9721a44862f4e91063a87f58 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 10 Jul 2024 18:33:52 -0600 Subject: [PATCH 07/14] chore: Add Miri workflow (#636) * add miri workflow * fix * ignore some tests when miri is enabled * fix one safety warning and ignore some slow tests * fmt * prepare for review * add comments * update to reflect change in directory name --- .github/workflows/miri.yml | 49 +++++++++++++++++++ native/core/Cargo.toml | 1 + native/core/src/errors.rs | 9 ++++ .../execution/datafusion/expressions/cast.rs | 2 + .../datafusion/expressions/xxhash64.rs | 1 + .../execution/datafusion/shuffle_writer.rs | 1 + .../src/execution/datafusion/spark_hash.rs | 5 +- native/core/src/execution/kernels/temporal.rs | 4 ++ native/core/src/execution/sort.rs | 2 +- 9 files changed, 69 insertions(+), 5 deletions(-) create mode 100644 .github/workflows/miri.yml diff --git a/.github/workflows/miri.yml b/.github/workflows/miri.yml new file mode 100644 index 000000000..a07ecc35e --- /dev/null +++ b/.github/workflows/miri.yml @@ -0,0 +1,49 @@ +# 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. + +name: Run Miri Safety Checks + +on: + push: + paths-ignore: + - "doc/**" + - "docs/**" + - "**.md" + pull_request: + paths-ignore: + - "doc/**" + - "docs/**" + - "**.md" + # manual trigger + # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow + workflow_dispatch: + +jobs: + miri: + name: "Miri" + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Install Miri + run: | + rustup toolchain install nightly --component miri + rustup override set nightly + cargo miri setup + - name: Test with Miri + run: | + cd native + MIRIFLAGS="-Zmiri-disable-isolation" cargo miri test diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 3820bdd38..6432118d6 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -93,6 +93,7 @@ twox-hash = "1.6.3" [features] default = [] +nightly = [] [lib] name = "comet" diff --git a/native/core/src/errors.rs b/native/core/src/errors.rs index 7b0b57448..8c02a72d1 100644 --- a/native/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -586,6 +586,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn error_from_panic() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -604,6 +605,7 @@ mod tests { // Verify that functions that return an object are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn object_result() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -621,6 +623,7 @@ mod tests { // Verify that functions that return an native time are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_result() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -637,6 +640,7 @@ mod tests { // Verify that functions that return an array can handle throwing exceptions. The test // causes an exception by dividing by zero. #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_panic_exception() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -657,6 +661,7 @@ mod tests { // Verify that functions that return an native time are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_result_ok() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -673,6 +678,7 @@ mod tests { // Verify that functions that return an native time are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_result_err() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -693,6 +699,7 @@ mod tests { // Verify that functions that return an array are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jint_array_result() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -713,6 +720,7 @@ mod tests { // Verify that functions that return an array can handle throwing exceptions. The test // causes an exception by dividing by zero. #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jint_array_panic_exception() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -736,6 +744,7 @@ mod tests { /// See [`object_panic_exception`] for a test which involves generating a panic and verifying /// that the resulting stack trace includes the offending call. #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn stacktrace_string() { // Setup: Start with a backtrace that includes all of the expected scenarios, including // cases where the file and location are not provided as part of the backtrace capture diff --git a/native/core/src/execution/datafusion/expressions/cast.rs b/native/core/src/execution/datafusion/expressions/cast.rs index 9e3205cef..154ff28b5 100644 --- a/native/core/src/execution/datafusion/expressions/cast.rs +++ b/native/core/src/execution/datafusion/expressions/cast.rs @@ -1646,6 +1646,7 @@ mod tests { use super::*; #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn timestamp_parser_test() { // write for all formats assert_eq!( @@ -1683,6 +1684,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_cast_string_to_timestamp() { let array: ArrayRef = Arc::new(StringArray::from(vec![ Some("2020-01-01T12:34:56.123456"), diff --git a/native/core/src/execution/datafusion/expressions/xxhash64.rs b/native/core/src/execution/datafusion/expressions/xxhash64.rs index 94b9e04ba..508cfe59b 100644 --- a/native/core/src/execution/datafusion/expressions/xxhash64.rs +++ b/native/core/src/execution/datafusion/expressions/xxhash64.rs @@ -162,6 +162,7 @@ mod test { use twox_hash::XxHash64; #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_xxhash64_random() { let mut rng = rand::thread_rng(); for len in 0..128 { diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs index 6e59ce53a..3b934813d 100644 --- a/native/core/src/execution/datafusion/shuffle_writer.rs +++ b/native/core/src/execution/datafusion/shuffle_writer.rs @@ -1451,6 +1451,7 @@ mod test { } #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx` fn test_insert_larger_batch() { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); let mut b = StringBuilder::new(); diff --git a/native/core/src/execution/datafusion/spark_hash.rs b/native/core/src/execution/datafusion/spark_hash.rs index 1f0658aec..15bded3a8 100644 --- a/native/core/src/execution/datafusion/spark_hash.rs +++ b/native/core/src/execution/datafusion/spark_hash.rs @@ -89,10 +89,7 @@ pub(crate) fn spark_compatible_murmur3_hash>(data: T, seed: u32) // data is &[u8] so we do not need to check for proper alignment unsafe { let mut h1 = if len_aligned > 0 { - hash_bytes_by_int( - std::slice::from_raw_parts(data.get_unchecked(0), len_aligned), - seed, - ) + hash_bytes_by_int(&data[0..len_aligned], seed) } else { seed as i32 }; diff --git a/native/core/src/execution/kernels/temporal.rs b/native/core/src/execution/kernels/temporal.rs index 1868c6fe5..9cf35af1a 100644 --- a/native/core/src/execution/kernels/temporal.rs +++ b/native/core/src/execution/kernels/temporal.rs @@ -824,6 +824,7 @@ mod tests { use std::sync::Arc; #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_date_trunc() { let size = 1000; let mut vec: Vec = Vec::with_capacity(size); @@ -962,6 +963,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_timestamp_trunc() { let size = 1000; let mut vec: Vec = Vec::with_capacity(size); @@ -998,6 +1000,8 @@ mod tests { } #[test] + // test takes too long with miri + #[cfg_attr(miri, ignore)] // This test only verifies that the various input array types work. Actually correctness to // ensure this produces the same results as spark is verified in the JVM tests fn test_timestamp_trunc_array_fmt_dyn() { diff --git a/native/core/src/execution/sort.rs b/native/core/src/execution/sort.rs index eeeb11d5b..b8687652c 100644 --- a/native/core/src/execution/sort.rs +++ b/native/core/src/execution/sort.rs @@ -165,7 +165,7 @@ where // because they are defined as Vec> ptr::copy_nonoverlapping( bucket.as_ptr(), - self.get_unchecked_mut(pos), + self.as_mut_ptr().add(pos), bucket.len(), ); } From a6a45fc7d19572f5d866e8ae31cdd5ad0f415f21 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 11 Jul 2024 05:39:58 -0600 Subject: [PATCH 08/14] feat: Move `IfExpr` to `spark-expr` crate (#653) --- native/Cargo.lock | 10 ++++++ native/Cargo.toml | 5 ++- native/core/Cargo.toml | 3 +- .../execution/datafusion/expressions/mod.rs | 1 - .../execution/datafusion/expressions/utils.rs | 18 ++-------- .../core/src/execution/datafusion/planner.rs | 3 +- native/spark-expr/Cargo.toml | 2 ++ .../expressions => spark-expr/src}/if_expr.rs | 20 +++++------ native/spark-expr/src/lib.rs | 6 +++- native/utils/Cargo.toml | 34 ++++++++++++++++++ native/utils/README.md | 22 ++++++++++++ native/utils/src/lib.rs | 36 +++++++++++++++++++ 12 files changed, 128 insertions(+), 32 deletions(-) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/if_expr.rs (95%) create mode 100644 native/utils/Cargo.toml create mode 100644 native/utils/README.md create mode 100644 native/utils/src/lib.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index c0f22fa1a..9bf8247d0 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -867,6 +867,7 @@ dependencies = [ "criterion", "datafusion", "datafusion-comet-spark-expr", + "datafusion-comet-utils", "datafusion-common", "datafusion-expr", "datafusion-physical-expr", @@ -909,8 +910,17 @@ dependencies = [ "arrow", "arrow-schema", "datafusion", + "datafusion-comet-utils", "datafusion-common", "datafusion-functions", + "datafusion-physical-expr", +] + +[[package]] +name = "datafusion-comet-utils" +version = "0.1.0" +dependencies = [ + "datafusion-physical-plan", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index 13860fbdf..53afed85a 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -16,7 +16,7 @@ # under the License. [workspace] -members = ["core", "spark-expr"] +members = ["core", "spark-expr", "utils"] resolver = "2" [workspace.package] @@ -43,8 +43,11 @@ datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "4 datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["unicode_expressions", "crypto_expressions"] } datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["crypto_expressions"] } datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } +datafusion-comet-spark-expr = { path = "spark-expr", version = "0.1.0" } +datafusion-comet-utils = { path = "utils", version = "0.1.0" } [profile.release] debug = true diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 6432118d6..be135d4e9 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -77,7 +77,8 @@ once_cell = "1.18.0" regex = "1.9.6" crc32fast = "1.3.2" simd-adler32 = "0.3.7" -datafusion-comet-spark-expr = { path = "../spark-expr", version = "0.1.0" } +datafusion-comet-spark-expr = { workspace = true } +datafusion-comet-utils = { workspace = true } [build-dependencies] prost-build = "0.9.0" diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 98b422dce..d573c2377 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -20,7 +20,6 @@ pub mod bitwise_not; pub mod cast; pub mod checkoverflow; -pub mod if_expr; mod normalize_nan; pub mod scalar_funcs; pub use normalize_nan::NormalizeNaNAndZero; diff --git a/native/core/src/execution/datafusion/expressions/utils.rs b/native/core/src/execution/datafusion/expressions/utils.rs index ee8646a78..6a7ec2e12 100644 --- a/native/core/src/execution/datafusion/expressions/utils.rs +++ b/native/core/src/execution/datafusion/expressions/utils.rs @@ -30,24 +30,10 @@ use arrow_array::{cast::AsArray, types::ArrowPrimitiveType}; use arrow_schema::DataType; use chrono::{DateTime, Offset, TimeZone}; use datafusion_common::cast::as_generic_string_array; -use datafusion_physical_expr::PhysicalExpr; use num::integer::div_floor; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; -/// An utility function from DataFusion. It is not exposed by DataFusion. -pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else { - any - } -} +pub use datafusion_comet_utils::down_cast_any_ref; /// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or /// to apply timezone offset. diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index ee208ac74..23960c307 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -79,7 +79,6 @@ use crate::{ checkoverflow::CheckOverflow, correlation::Correlation, covariance::Covariance, - if_expr::IfExpr, negative, scalar_funcs::create_comet_physical_fun, stats::StatsType, @@ -108,7 +107,7 @@ use crate::{ }; use super::expressions::{create_named_struct::CreateNamedStruct, EvalMode}; -use datafusion_comet_spark_expr::abs::Abs; +use datafusion_comet_spark_expr::{Abs, IfExpr}; // For clippy error on type_complexity. type ExecResult = Result; diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index d10d04944..8bf76dff6 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -32,6 +32,8 @@ arrow-schema = { workspace = true } datafusion = { workspace = true } datafusion-common = { workspace = true } datafusion-functions = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-comet-utils = { workspace = true } [lib] name = "datafusion_comet_spark_expr" diff --git a/native/core/src/execution/datafusion/expressions/if_expr.rs b/native/spark-expr/src/if_expr.rs similarity index 95% rename from native/core/src/execution/datafusion/expressions/if_expr.rs rename to native/spark-expr/src/if_expr.rs index fa235cc66..c04494ec4 100644 --- a/native/core/src/execution/datafusion/expressions/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -31,7 +31,7 @@ use datafusion::logical_expr::ColumnarValue; use datafusion_common::{cast::as_boolean_array, Result}; use datafusion_physical_expr::PhysicalExpr; -use crate::execution::datafusion::expressions::utils::down_cast_any_ref; +use datafusion_comet_utils::down_cast_any_ref; #[derive(Debug, Hash)] pub struct IfExpr { @@ -147,15 +147,6 @@ impl PartialEq for IfExpr { } } -/// Create an If expression -pub fn if_fn( - if_expr: Arc, - true_expr: Arc, - false_expr: Arc, -) -> Result> { - Ok(Arc::new(IfExpr::new(if_expr, true_expr, false_expr))) -} - #[cfg(test)] mod tests { use arrow::{array::StringArray, datatypes::*}; @@ -165,6 +156,15 @@ mod tests { use super::*; + /// Create an If expression + fn if_fn( + if_expr: Arc, + true_expr: Arc, + false_expr: Arc, + ) -> Result> { + Ok(Arc::new(IfExpr::new(if_expr, true_expr, false_expr))) + } + #[test] fn test_if_1() -> Result<()> { let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 3873754be..c36e8855e 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -18,7 +18,11 @@ use std::error::Error; use std::fmt::{Display, Formatter}; -pub mod abs; +mod abs; +mod if_expr; + +pub use abs::Abs; +pub use if_expr::IfExpr; /// Spark supports three evaluation modes when evaluating expressions, which affect /// the behavior when processing input values that are invalid or would result in an diff --git a/native/utils/Cargo.toml b/native/utils/Cargo.toml new file mode 100644 index 000000000..05ddd3488 --- /dev/null +++ b/native/utils/Cargo.toml @@ -0,0 +1,34 @@ +# 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. + +[package] +name = "datafusion-comet-utils" +description = "DataFusion Comet Utilities" +version = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +authors = { workspace = true } +readme = { workspace = true } +license = { workspace = true } +edition = { workspace = true } + +[dependencies] +datafusion-physical-plan = { workspace = true } + +[lib] +name = "datafusion_comet_utils" +path = "src/lib.rs" diff --git a/native/utils/README.md b/native/utils/README.md new file mode 100644 index 000000000..513c6245e --- /dev/null +++ b/native/utils/README.md @@ -0,0 +1,22 @@ + + +# datafusion-comet-utils + +This crate provides utilities for use in the [Apache DataFusion Comet](https://github.com/apache/datafusion-comet/) project. \ No newline at end of file diff --git a/native/utils/src/lib.rs b/native/utils/src/lib.rs new file mode 100644 index 000000000..54ff55b46 --- /dev/null +++ b/native/utils/src/lib.rs @@ -0,0 +1,36 @@ +// 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::any::Any; +use std::sync::Arc; + +use datafusion_physical_plan::PhysicalExpr; + +/// A utility function from DataFusion. It is not exposed by DataFusion. +pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { + if any.is::>() { + any.downcast_ref::>() + .unwrap() + .as_any() + } else if any.is::>() { + any.downcast_ref::>() + .unwrap() + .as_any() + } else { + any + } +} From 8274a618127123dbafa0954fac0193fea43dc116 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 11 Jul 2024 10:46:12 -0700 Subject: [PATCH 09/14] test: Run optimized version of q72 derived from TPC-DS (#652) --- .../src/test/resources/tpcds-extended/q72.sql | 26 +++++ .../tpcds-query-results/extended/q72.sql.out | 105 ++++++++++++++++++ .../spark/sql/CometTPCDSQueryTestSuite.scala | 15 +++ 3 files changed, 146 insertions(+) create mode 100644 spark/src/test/resources/tpcds-extended/q72.sql create mode 100644 spark/src/test/resources/tpcds-query-results/extended/q72.sql.out diff --git a/spark/src/test/resources/tpcds-extended/q72.sql b/spark/src/test/resources/tpcds-extended/q72.sql new file mode 100644 index 000000000..4c95c8aa7 --- /dev/null +++ b/spark/src/test/resources/tpcds-extended/q72.sql @@ -0,0 +1,26 @@ +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join item on (i_item_sk = cs_item_sk) +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and d3.d_date > d1.d_date + 5 + and hd_buy_potential = '501-1000' + and d1.d_year = 1999 + and cd_marital_status = 'S' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +LIMIT 100 diff --git a/spark/src/test/resources/tpcds-query-results/extended/q72.sql.out b/spark/src/test/resources/tpcds-query-results/extended/q72.sql.out new file mode 100644 index 000000000..7054939ef --- /dev/null +++ b/spark/src/test/resources/tpcds-query-results/extended/q72.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by CometTPCDSQuerySuite + +-- !query schema +struct +-- !query output +Acceptable statements kill only important grounds. Various, high rights can lie also maj Operations 5197 0 2 2 +Acceptable statements kill only important grounds. Various, high rights can lie also maj Selective, 5197 0 2 2 +Bloody conditions cannot become. Libraries operate enough further good police. Matches produce 5212 0 2 2 +Democratic friends would represent. British, welcome problems could show never better special classes. Gradually daily sites practice at a men. Genetic, british numbers use Matches produce 5215 0 2 2 +Direct, private interpretations look perhaps under political policies. Casualties can feel only results. Architects Matches produce 5197 0 2 2 +Due conclusions will not attra Just good amou 5213 0 2 2 +Free activities pay particularly from a questions. British engineers complete commonly political doors. Bright, w Operations 5179 0 2 2 +Great doors could recall too studies. Then dual things kill. Meanwhile interesting appointments should name direct months. Potential theories change constant feet; hot resourc Selective, 5179 0 2 2 +Grey artists think there. Points meet with a p Just good amou 5197 0 2 2 +Here economic words appear widely between a feet. Financial yards used to wo Selective, 5215 0 2 2 +Ideas would allow full-time good m Just good amou 5197 0 2 2 +Images will relax in particular new skills. Urban obligations must not mark completely. Officials must not cope both emotionally dif Just good amou 5195 0 2 2 +Images will relax in particular new skills. Urban obligations must not mark completely. Officials must not cope both emotionally dif Operations 5195 0 2 2 +Other parties will add regional, special owners. Little administrative horses may indicate; Matches produce 5206 0 2 2 +Parliamentary, irish achievements come specifically probably normal efforts. Boxes prove activities. Particular spirits display above, hot legs. Willing, warm police w Matches produce 5212 0 2 2 +Patterns hear available, bitter women. Certain, standard proposals m Operations 5205 0 2 2 +Years w Matches produce 5211 0 2 2 +NULL Just good amou 5211 0 1 1 +NULL Matches produce 5202 0 1 1 +NULL Matches produce 5209 0 1 1 +NULL Operations 5198 0 1 1 +NULL Operations 5209 0 1 1 +NULL Operations 5217 0 1 1 +NULL Selective, 5181 0 1 1 +NULL Significantly 5209 0 1 1 +A Selective, 5194 0 1 1 +A Selective, 5208 0 1 1 +A little local letters think over like a children; nevertheless particular powers damage now suddenly absent prote Matches produce 5215 0 1 1 +A little local letters think over like a children; nevertheless particular powers damage now suddenly absent prote Significantly 5205 0 1 1 +A lot young materials remain below from a rises. Operations 5186 0 1 1 +Able men become in order today other methods. Slightly royal shoulders would not operate centres. Statutory, social bills may not take other, main stages. Prayers cannot meet. Words m Selective, 5177 0 1 1 +Able men become in order today other methods. Slightly royal shoulders would not operate centres. Statutory, social bills may not take other, main stages. Prayers cannot meet. Words m Significantly 5177 0 1 1 +Able troubles dust into the styles. Independent feet kill wounds. Fundamental months should exploit arms. Massive years read only modern courses; twin forms shall become products. Even h Significantly 5215 0 1 1 +Able, available problems apply even to a bodies. Patients so Matches produce 5211 0 1 1 +Able, continuous figures see with a patients. Men go more open notes. Different engineers can display. Even strong fortunes cannot put at least general fans; reliable talk Matches produce 5211 0 1 1 +Able, continuous figures see with a patients. Men go more open notes. Different engineers can display. Even strong fortunes cannot put at least general fans; reliable talk Operations 5211 0 1 1 +About right clothes must get thoughtfully to a cases. Eastern improvements Matches produce 5202 0 1 1 +Ac Just good amou 5205 0 1 1 +Ac Matches produce 5183 0 1 1 +Accessible kids insist ministers. Causal, political goods find then other products. Following, average years may want considerably now general causes. N Selective, 5209 0 1 1 +Accidentally wrong communities look more goods. Rural matters recognize. Large, new days go hap Selective, 5178 0 1 1 +Accidents fly bet Significantly 5167 0 1 1 +Accounts accept Operations 5213 0 1 1 +Accounts could think aspects. Industrial, large Selective, 5211 0 1 1 +Accounts return into a colleagues Operations 5176 0 1 1 +Accused times should stop then notes. Wor Just good amou 5190 0 1 1 +Accused times should stop then notes. Wor Selective, 5218 0 1 1 +Acres could provide. There chronic principles care highest at a details. Little operators look elsewhere long recommendations. Daily poets dig bravely remaining aims. Now willing services appear Selective, 5205 0 1 1 +Actions see of course informal phrases. Markedly right men buy honest, additional stations. In order imaginative factors used to move human thanks. Centres shall catch altogether succe Just good amou 5192 0 1 1 +Active studies state away already large shelves. Extremely international appli Significantly 5208 0 1 1 +Active years refuse all poor others. Old, reluctant friends live new, liberal varieties. English, full hours belong as eve Matches produce 5202 0 1 1 +Acts protect forces. Still only options use controls. Less manufacturing cups let well sure, good fires. Sorry companies should die. All rare nurses maintain Just good amou 5212 0 1 1 +Acts protect forces. Still only options use controls. Less manufacturing cups let well sure, good fires. Sorry companies should die. All rare nurses maintain Selective, 5200 0 1 1 +Acts will not reflect as with the problems. General governments distract new, soft fires. Useful proposals restrict hard trees. Large, black customs go official Just good amou 5201 0 1 1 +Actual things should prevent there responsible schemes. Others go all undoubtedly increasing things. Small, full samples analys Significantly 5167 0 1 1 +Actually labour schools think about a holes. So disciplinary prov Significantly 5179 0 1 1 +Acute, old funds must sleep running, different orders. Black, international changes should cope enough multiple brothers. Ago available police attract simply internationa Matches produce 5207 0 1 1 +Additional, human standards should not dream also silly forms. More independent friends may believ Significantly 5203 0 1 1 +Additionally effective sea Operations 5182 0 1 1 +Addresses retain once more applicable events. Following blocks follow for a develo Selective, 5169 0 1 1 +Adequate days take more sometimes high laws. Famous, exceptional gates must pay loans. Other, slow methods see superb aspirations. Possible services apply holes. Computers can free sole times. Costly Matches produce 5201 0 1 1 +Adults might not surrender doubtful, upper industries; earnings insist m Operations 5181 0 1 1 +Adults will foresee most left, social children. Different eyes make personal counties. Readers would not admit more musical proceedings; titles take here away fast institutions; bird Significantly 5215 0 1 1 +Advanced, foreign stories would greet always corporate games. Recent dev Significantly 5182 0 1 1 +Again brief things should remember only in a patients. Deals reply soon other points. Increasingly religious times necessitate farther troops. Both added programmes must come wonderfully solid pupi Matches produce 5212 0 1 1 +Again brief things should remember only in a patients. Deals reply soon other points. Increasingly religious times necessitate farther troops. Both added programmes must come wonderfully solid pupi Operations 5212 0 1 1 +Again economic objections spend suddenly urgently worried boats. Pupils talk yet nonethele Significantly 5205 0 1 1 +Again major troubles create new, other children. Fair interactions may telephone Matches produce 5188 0 1 1 +Again scottish women shall ag Just good amou 5204 0 1 1 +Again small aspects think apparently only sole problems; thick private years may not matt Operations 5197 0 1 1 +Agencies join enough keen examples. Accu Just good amou 5214 0 1 1 +Ago light fingers blame enough green, british years. Children go general stands. Economic, great numbers affect deputies. Purposes urge annually. Always electrical ways vote judicial, regular ac Just good amou 5196 0 1 1 +Ago light fingers blame enough green, british years. Children go general stands. Economic, great numbers affect deputies. Purposes urge annually. Always electrical ways vote judicial, regular ac Just good amou 5208 0 1 1 +Ago new arguments accept previously european parents; fo Matches produce 5177 0 1 1 +Ago new arguments accept previously european parents; fo Significantly 5177 0 1 1 +Agricultural players shall smoke. So full reasons undertake Just good amou 5218 0 1 1 +Agricultural sites will not provide skills. Again Operations 5217 0 1 1 +All possible injuries maintain strictly mu Significantly 5211 0 1 1 +All right other details might distrib Operations 5186 0 1 1 +Almost little words used to explain annual men. Opportunities borrow. Quickly general alternatives see prices. Again green firms must not help most so able authorities; thankfully married hours mig Just good amou 5204 0 1 1 +Almost positive couples draw into a orders. Glasses adopt shortly mathematical years. Ever big women set details. Successful, joint factors shall allow never at all overseas deals. Topics would use n Significantly 5210 0 1 1 +Almost scientific flats divide easy, able hours. Broad years sit also through a circumstances; more Just good amou 5181 0 1 1 +Alone healthy sales might meet far other roots. French groups look up to a workers. Fully average miners would walk inadequate considerations. Small, sure goods may admire more app Matches produce 5207 0 1 1 +Alone new copies discuss to a dates; all black machines get just just royal years. For example free weeks underestimate accurately individual mountains. National, delicious Just good amou 5211 0 1 1 +Alone sole services keep only; stairs shall eliminate for the woods. Methods must need yet. Other students can Matches produce 5199 0 1 1 +Already early meetings cannot go animals. As comprehensive evenings w Selective, 5201 0 1 1 +Already empty cups would prove already other white times. Various, able hands live here different countries. Dealers may not cut effectively important institutions. Rich, dramatic t Selective, 5214 0 1 1 +Already little levels will assume specialist, main costs. By now new miles play meanwhile men; common, economic results achieve furious, sure holes; persistent men die Operations 5218 0 1 1 +Also aware programmes should send for certain white letters. Eyes would not treat visible centuries; wide, full languages see on behalf of the ages. Guards illuminate feet. Nationally criminal cou Significantly 5191 0 1 1 +Also fun tories test of course. Rigid walls indicate then doors. Sometimes large children want just foreign, americ Matches produce 5169 0 1 1 +Also major pieces resign never. Substan Matches produce 5178 0 1 1 +Also middle managers can appear originally negotiations. Things must stand distinguished funds; only customers find here also foreign seeds. Green, political Operations 5202 0 1 1 +Also possible systems could go forward. Local, british babies d Matches produce 5215 0 1 1 +Also possible systems could go forward. Local, british babies d Selective, 5215 0 1 1 +Also public dreams see. Then semantic services can allow in a powers. Things must make sections. Spaces would not protect officials. Inappropriate charges would act r Matches produce 5177 0 1 1 +Also quiet users fall. Other, current sources would c Operations 5206 0 1 1 +Also rational children achieve other trends. Rather conventional commitments ought to refuse then current olympic members; suitably special te Just good amou 5210 0 1 1 +Also rational children achieve other trends. Rather conventional commitments ought to refuse then current olympic members; suitably special te Selective, 5210 0 1 1 +Alternative sessions might not know here all literary organisers. Ho Significantly 5213 0 1 1 +Always cool temperatures meet there social grounds. Threats h Matches produce 5175 0 1 1 diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala index c2b853515..6aec042af 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala @@ -35,6 +35,8 @@ import org.apache.spark.sql.test.TestSparkSession */ class CometTPCDSQueryTestSuite extends QueryTest with TPCDSBase with CometSQLQueryTestHelper { + val tpcdsExtendedQueries: Seq[String] = Seq("q72") + private val tpcdsDataPath = sys.env.get("SPARK_TPCDS_DATA") private val regenGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" @@ -224,6 +226,19 @@ class CometTPCDSQueryTestSuite extends QueryTest with TPCDSBase with CometSQLQue } } } + + tpcdsExtendedQueries.foreach { name => + val queryString = resourceToString( + s"tpcds-extended/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(s"extended $name") { + val goldenFile = new File(s"$baseResourcePath/extended", s"$name.sql.out") + joinConfs.foreach { conf => + System.gc() // SPARK-37368 + runQuery(queryString, goldenFile, conf) + } + } + } } else { ignore("skipped because env `SPARK_TPCDS_DATA` is not set") {} } From 863b40f5f99f4116ad627accbc863fecb9017d50 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 12 Jul 2024 05:52:16 -0600 Subject: [PATCH 10/14] chore: Refactoring of CometError/SparkError (#655) --- native/Cargo.lock | 1 + native/Cargo.toml | 1 + native/core/Cargo.toml | 2 +- native/core/src/errors.rs | 41 +----- .../execution/datafusion/expressions/cast.rs | 126 +++++++++--------- .../execution/datafusion/expressions/mod.rs | 6 +- .../datafusion/expressions/negative.rs | 6 +- native/spark-expr/Cargo.toml | 1 + native/spark-expr/src/abs.rs | 7 +- native/spark-expr/src/error.rs | 73 ++++++++++ native/spark-expr/src/lib.rs | 21 +-- 11 files changed, 157 insertions(+), 128 deletions(-) create mode 100644 native/spark-expr/src/error.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 9bf8247d0..605af92ee 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -914,6 +914,7 @@ dependencies = [ "datafusion-common", "datafusion-functions", "datafusion-physical-expr", + "thiserror", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index 53afed85a..944b6e28a 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -48,6 +48,7 @@ datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion. datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.1.0" } datafusion-comet-utils = { path = "utils", version = "0.1.0" } +thiserror = "1" [profile.release] debug = true diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index be135d4e9..50c1ce2b3 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -48,7 +48,7 @@ tokio = { version = "1", features = ["rt-multi-thread"] } async-trait = "0.1" log = "0.4" log4rs = "1.2.0" -thiserror = "1" +thiserror = { workspace = true } serde = { version = "1", features = ["derive"] } lazy_static = "1.4.0" prost = "0.12.1" diff --git a/native/core/src/errors.rs b/native/core/src/errors.rs index 8c02a72d1..ff89e77d2 100644 --- a/native/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -38,6 +38,7 @@ use std::{ use jni::sys::{jboolean, jbyte, jchar, jdouble, jfloat, jint, jlong, jobject, jshort}; use crate::execution::operators::ExecutionError; +use datafusion_comet_spark_expr::SparkError; use jni::objects::{GlobalRef, JThrowable}; use jni::JNIEnv; use lazy_static::lazy_static; @@ -62,36 +63,10 @@ pub enum CometError { #[error("Comet Internal Error: {0}")] Internal(String), - // Note that this message format is based on Spark 3.4 and is more detailed than the message - // returned by Spark 3.3 - #[error("[CAST_INVALID_INPUT] The value '{value}' of the type \"{from_type}\" cannot be cast to \"{to_type}\" \ - because it is malformed. Correct the value as per the syntax, or change its target type. \ - Use `try_cast` to tolerate malformed input and return NULL instead. If necessary \ - set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] - CastInvalidValue { - value: String, - from_type: String, - to_type: String, - }, - - #[error("[NUMERIC_VALUE_OUT_OF_RANGE] {value} cannot be represented as Decimal({precision}, {scale}). If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error, and return NULL instead.")] - NumericValueOutOfRange { - value: String, - precision: u8, - scale: i8, - }, - - #[error("[CAST_OVERFLOW] The value {value} of the type \"{from_type}\" cannot be cast to \"{to_type}\" \ - due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary \ - set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] - CastOverFlow { - value: String, - from_type: String, - to_type: String, - }, - - #[error("[ARITHMETIC_OVERFLOW] {from_type} overflow. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] - ArithmeticOverflow { from_type: String }, + /// CometError::Spark is typically used in native code to emulate the same errors + /// that Spark would return + #[error(transparent)] + Spark(SparkError), #[error(transparent)] Arrow { @@ -239,11 +214,7 @@ impl jni::errors::ToException for CometError { class: "java/lang/NullPointerException".to_string(), msg: self.to_string(), }, - CometError::CastInvalidValue { .. } => Exception { - class: "org/apache/spark/SparkException".to_string(), - msg: self.to_string(), - }, - CometError::NumericValueOutOfRange { .. } => Exception { + CometError::Spark { .. } => Exception { class: "org/apache/spark/SparkException".to_string(), msg: self.to_string(), }, diff --git a/native/core/src/execution/datafusion/expressions/cast.rs b/native/core/src/execution/datafusion/expressions/cast.rs index 154ff28b5..0b513e776 100644 --- a/native/core/src/execution/datafusion/expressions/cast.rs +++ b/native/core/src/execution/datafusion/expressions/cast.rs @@ -40,16 +40,14 @@ use arrow_array::{ use arrow_schema::{DataType, Schema}; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; use datafusion::logical_expr::ColumnarValue; +use datafusion_comet_spark_expr::{SparkError, SparkResult}; use datafusion_common::{internal_err, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use num::{cast::AsPrimitive, traits::CheckedNeg, CheckedSub, Integer, Num, ToPrimitive}; use regex::Regex; -use crate::{ - errors::{CometError, CometResult}, - execution::datafusion::expressions::utils::{ - array_with_timezone, down_cast_any_ref, spark_cast, - }, +use crate::execution::datafusion::expressions::utils::{ + array_with_timezone, down_cast_any_ref, spark_cast, }; use super::EvalMode; @@ -87,7 +85,7 @@ macro_rules! cast_utf8_to_int { cast_array.append_null() } } - let result: CometResult = Ok(Arc::new(cast_array.finish()) as ArrayRef); + let result: SparkResult = Ok(Arc::new(cast_array.finish()) as ArrayRef); result }}; } @@ -116,7 +114,7 @@ macro_rules! cast_float_to_string { fn cast( from: &dyn Array, _eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { let array = from.as_any().downcast_ref::<$output_type>().unwrap(); @@ -169,7 +167,7 @@ macro_rules! cast_float_to_string { Some(value) => Ok(Some(value.to_string())), _ => Ok(None), }) - .collect::, CometError>>()?; + .collect::, SparkError>>()?; Ok(Arc::new(output_array)) } @@ -205,7 +203,7 @@ macro_rules! cast_int_to_int_macro { .iter() .map(|value| match value { Some(value) => { - Ok::, CometError>(Some(value as $to_native_type)) + Ok::, SparkError>(Some(value as $to_native_type)) } _ => Ok(None), }) @@ -222,14 +220,14 @@ macro_rules! cast_int_to_int_macro { $spark_to_data_type_name, )) } else { - Ok::, CometError>(Some(res.unwrap())) + Ok::, SparkError>(Some(res.unwrap())) } } _ => Ok(None), }) .collect::, _>>(), }?; - let result: CometResult = Ok(Arc::new(output_array) as ArrayRef); + let result: SparkResult = Ok(Arc::new(output_array) as ArrayRef); result }}; } @@ -286,7 +284,7 @@ macro_rules! cast_float_to_int16_down { .map(|value| match value { Some(value) => { let i32_value = value as i32; - Ok::, CometError>(Some( + Ok::, SparkError>(Some( i32_value as $rust_dest_type, )) } @@ -339,7 +337,7 @@ macro_rules! cast_float_to_int32_up { .iter() .map(|value| match value { Some(value) => { - Ok::, CometError>(Some(value as $rust_dest_type)) + Ok::, SparkError>(Some(value as $rust_dest_type)) } None => Ok(None), }) @@ -402,7 +400,7 @@ macro_rules! cast_decimal_to_int16_down { Some(value) => { let divisor = 10_i128.pow($scale as u32); let i32_value = (value / divisor) as i32; - Ok::, CometError>(Some( + Ok::, SparkError>(Some( i32_value as $rust_dest_type, )) } @@ -456,7 +454,7 @@ macro_rules! cast_decimal_to_int32_up { Some(value) => { let divisor = 10_i128.pow($scale as u32); let truncated = value / divisor; - Ok::, CometError>(Some( + Ok::, SparkError>(Some( truncated as $rust_dest_type, )) } @@ -596,7 +594,7 @@ impl Cast { // we should never reach this code because the Scala code should be checking // for supported cast operations and falling back to Spark for anything that // is not yet supported - Err(CometError::Internal(format!( + Err(SparkError::Internal(format!( "Native cast invoked for unsupported cast from {from_type:?} to {to_type:?}" ))) } @@ -680,7 +678,7 @@ impl Cast { to_type: &DataType, array: &ArrayRef, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { let string_array = array .as_any() .downcast_ref::>() @@ -711,7 +709,7 @@ impl Cast { array: &ArrayRef, to_type: &DataType, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { let string_array = array .as_any() .downcast_ref::>() @@ -743,7 +741,7 @@ impl Cast { array: &ArrayRef, to_type: &DataType, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { let string_array = array .as_any() .downcast_ref::>() @@ -768,7 +766,7 @@ impl Cast { precision: u8, scale: i8, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { Self::cast_floating_point_to_decimal128::(array, precision, scale, eval_mode) } @@ -777,7 +775,7 @@ impl Cast { precision: u8, scale: i8, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { Self::cast_floating_point_to_decimal128::(array, precision, scale, eval_mode) } @@ -786,7 +784,7 @@ impl Cast { precision: u8, scale: i8, eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where ::Native: AsPrimitive, { @@ -806,7 +804,7 @@ impl Cast { Some(v) => { if Decimal128Type::validate_decimal_precision(v, precision).is_err() { if eval_mode == EvalMode::Ansi { - return Err(CometError::NumericValueOutOfRange { + return Err(SparkError::NumericValueOutOfRange { value: input_value.to_string(), precision, scale, @@ -819,7 +817,7 @@ impl Cast { } None => { if eval_mode == EvalMode::Ansi { - return Err(CometError::NumericValueOutOfRange { + return Err(SparkError::NumericValueOutOfRange { value: input_value.to_string(), precision, scale, @@ -843,7 +841,7 @@ impl Cast { fn spark_cast_float64_to_utf8( from: &dyn Array, _eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { @@ -853,7 +851,7 @@ impl Cast { fn spark_cast_float32_to_utf8( from: &dyn Array, _eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { @@ -865,7 +863,7 @@ impl Cast { eval_mode: EvalMode, from_type: &DataType, to_type: &DataType, - ) -> CometResult { + ) -> SparkResult { match (from_type, to_type) { (DataType::Int64, DataType::Int32) => cast_int_to_int_macro!( array, eval_mode, Int64Type, Int32Type, from_type, i32, "BIGINT", "INT" @@ -895,7 +893,7 @@ impl Cast { fn spark_cast_utf8_to_boolean( from: &dyn Array, eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { @@ -910,7 +908,7 @@ impl Cast { Some(value) => match value.to_ascii_lowercase().trim() { "t" | "true" | "y" | "yes" | "1" => Ok(Some(true)), "f" | "false" | "n" | "no" | "0" => Ok(Some(false)), - _ if eval_mode == EvalMode::Ansi => Err(CometError::CastInvalidValue { + _ if eval_mode == EvalMode::Ansi => Err(SparkError::CastInvalidValue { value: value.to_string(), from_type: "STRING".to_string(), to_type: "BOOLEAN".to_string(), @@ -929,7 +927,7 @@ impl Cast { eval_mode: EvalMode, from_type: &DataType, to_type: &DataType, - ) -> CometResult { + ) -> SparkResult { match (from_type, to_type) { (DataType::Float32, DataType::Int8) => cast_float_to_int16_down!( array, @@ -1066,7 +1064,7 @@ impl Cast { } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toByte -fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> SparkResult> { Ok(cast_string_to_int_with_range_check( str, eval_mode, @@ -1078,7 +1076,7 @@ fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> CometResult> } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toShort -fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> SparkResult> { Ok(cast_string_to_int_with_range_check( str, eval_mode, @@ -1090,12 +1088,12 @@ fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> CometResult } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper) -fn cast_string_to_i32(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i32(str: &str, eval_mode: EvalMode) -> SparkResult> { do_cast_string_to_int::(str, eval_mode, "INT", i32::MIN) } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper intWrapper) -fn cast_string_to_i64(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i64(str: &str, eval_mode: EvalMode) -> SparkResult> { do_cast_string_to_int::(str, eval_mode, "BIGINT", i64::MIN) } @@ -1105,7 +1103,7 @@ fn cast_string_to_int_with_range_check( type_name: &str, min: i32, max: i32, -) -> CometResult> { +) -> SparkResult> { match do_cast_string_to_int(str, eval_mode, type_name, i32::MIN)? { None => Ok(None), Some(v) if v >= min && v <= max => Ok(Some(v)), @@ -1124,7 +1122,7 @@ fn do_cast_string_to_int< eval_mode: EvalMode, type_name: &str, min_value: T, -) -> CometResult> { +) -> SparkResult> { let trimmed_str = str.trim(); if trimmed_str.is_empty() { return none_or_err(eval_mode, type_name, str); @@ -1208,9 +1206,9 @@ fn do_cast_string_to_int< Ok(Some(result)) } -/// Either return Ok(None) or Err(CometError::CastInvalidValue) depending on the evaluation mode +/// Either return Ok(None) or Err(SparkError::CastInvalidValue) depending on the evaluation mode #[inline] -fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> CometResult> { +fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> SparkResult> { match eval_mode { EvalMode::Ansi => Err(invalid_value(str, "STRING", type_name)), _ => Ok(None), @@ -1218,8 +1216,8 @@ fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> CometResul } #[inline] -fn invalid_value(value: &str, from_type: &str, to_type: &str) -> CometError { - CometError::CastInvalidValue { +fn invalid_value(value: &str, from_type: &str, to_type: &str) -> SparkError { + SparkError::CastInvalidValue { value: value.to_string(), from_type: from_type.to_string(), to_type: to_type.to_string(), @@ -1227,8 +1225,8 @@ fn invalid_value(value: &str, from_type: &str, to_type: &str) -> CometError { } #[inline] -fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> CometError { - CometError::CastOverFlow { +fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> SparkError { + SparkError::CastOverFlow { value: value.to_string(), from_type: from_type.to_string(), to_type: to_type.to_string(), @@ -1316,7 +1314,7 @@ impl PhysicalExpr for Cast { } } -fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult> { +fn timestamp_parser(value: &str, eval_mode: EvalMode) -> SparkResult> { let value = value.trim(); if value.is_empty() { return Ok(None); @@ -1325,7 +1323,7 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult let patterns = &[ ( Regex::new(r"^\d{4}$").unwrap(), - parse_str_to_year_timestamp as fn(&str) -> CometResult>, + parse_str_to_year_timestamp as fn(&str) -> SparkResult>, ), ( Regex::new(r"^\d{4}-\d{2}$").unwrap(), @@ -1369,7 +1367,7 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult if timestamp.is_none() { return if eval_mode == EvalMode::Ansi { - Err(CometError::CastInvalidValue { + Err(SparkError::CastInvalidValue { value: value.to_string(), from_type: "STRING".to_string(), to_type: "TIMESTAMP".to_string(), @@ -1381,20 +1379,20 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult match timestamp { Some(ts) => Ok(Some(ts)), - None => Err(CometError::Internal( + None => Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )), } } -fn parse_ymd_timestamp(year: i32, month: u32, day: u32) -> CometResult> { +fn parse_ymd_timestamp(year: i32, month: u32, day: u32) -> SparkResult> { let datetime = chrono::Utc.with_ymd_and_hms(year, month, day, 0, 0, 0); // Check if datetime is not None let utc_datetime = match datetime.single() { Some(dt) => dt.with_timezone(&chrono::Utc), None => { - return Err(CometError::Internal( + return Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )); } @@ -1411,7 +1409,7 @@ fn parse_hms_timestamp( minute: u32, second: u32, microsecond: u32, -) -> CometResult> { +) -> SparkResult> { let datetime = chrono::Utc.with_ymd_and_hms(year, month, day, hour, minute, second); // Check if datetime is not None @@ -1420,7 +1418,7 @@ fn parse_hms_timestamp( .with_timezone(&chrono::Utc) .with_nanosecond(microsecond * 1000), None => { - return Err(CometError::Internal( + return Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )); } @@ -1429,7 +1427,7 @@ fn parse_hms_timestamp( let result = match utc_datetime { Some(dt) => dt.timestamp_micros(), None => { - return Err(CometError::Internal( + return Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )); } @@ -1438,7 +1436,7 @@ fn parse_hms_timestamp( Ok(Some(result)) } -fn get_timestamp_values(value: &str, timestamp_type: &str) -> CometResult> { +fn get_timestamp_values(value: &str, timestamp_type: &str) -> SparkResult> { let values: Vec<_> = value .split(|c| c == 'T' || c == '-' || c == ':' || c == '.') .collect(); @@ -1458,7 +1456,7 @@ fn get_timestamp_values(value: &str, timestamp_type: &str) -> CometResult