From 4d04df1bdbaa2dfac74065144ac5b4a268518899 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Wed, 18 Sep 2024 16:40:52 +0800 Subject: [PATCH] [GLUTEN-7203][CORE] Make push down filter to scan as an individual rule (#7215) Closes #7203 --- .../backendsapi/clickhouse/CHRuleApi.scala | 1 + .../clickhouse/CHSparkPlanExecApi.scala | 2 +- .../backendsapi/velox/VeloxRuleApi.scala | 2 + .../v1-bhj-ras/spark32/1.txt | 6 +- .../v1-bhj-ras/spark32/10.txt | 24 ++--- .../v1-bhj-ras/spark32/11.txt | 18 ++-- .../v1-bhj-ras/spark32/12.txt | 12 +-- .../v1-bhj-ras/spark32/13.txt | 6 +- .../v1-bhj-ras/spark32/14.txt | 12 +-- .../v1-bhj-ras/spark32/15.txt | 12 +-- .../v1-bhj-ras/spark32/16.txt | 12 +-- .../v1-bhj-ras/spark32/18.txt | 18 ++-- .../v1-bhj-ras/spark32/19.txt | 12 +-- .../v1-bhj-ras/spark32/20.txt | 30 +++---- .../v1-bhj-ras/spark32/21.txt | 30 +++---- .../v1-bhj-ras/spark32/22.txt | 6 +- .../v1-bhj-ras/spark32/3.txt | 18 ++-- .../v1-bhj-ras/spark32/4.txt | 12 +-- .../v1-bhj-ras/spark32/5.txt | 36 ++++---- .../v1-bhj-ras/spark32/6.txt | 6 +- .../v1-bhj-ras/spark32/7.txt | 30 +++---- .../v1-bhj-ras/spark32/8.txt | 48 +++++----- .../v1-bhj-ras/spark32/9.txt | 36 ++++---- .../v1-bhj-ras/spark33/1.txt | 6 +- .../v1-bhj-ras/spark33/10.txt | 24 ++--- .../v1-bhj-ras/spark33/11.txt | 24 ++--- .../v1-bhj-ras/spark33/12.txt | 12 +-- .../v1-bhj-ras/spark33/13.txt | 6 +- .../v1-bhj-ras/spark33/14.txt | 12 +-- .../v1-bhj-ras/spark33/15.txt | 18 ++-- .../v1-bhj-ras/spark33/16.txt | 12 +-- .../v1-bhj-ras/spark33/18.txt | 18 ++-- .../v1-bhj-ras/spark33/19.txt | 12 +-- .../v1-bhj-ras/spark33/20.txt | 30 +++---- .../v1-bhj-ras/spark33/21.txt | 30 +++---- .../v1-bhj-ras/spark33/22.txt | 37 ++++++-- .../v1-bhj-ras/spark33/3.txt | 18 ++-- .../v1-bhj-ras/spark33/4.txt | 12 +-- .../v1-bhj-ras/spark33/5.txt | 36 ++++---- .../v1-bhj-ras/spark33/6.txt | 6 +- .../v1-bhj-ras/spark33/7.txt | 30 +++---- .../v1-bhj-ras/spark33/8.txt | 48 +++++----- .../v1-bhj-ras/spark33/9.txt | 36 ++++---- .../v1-bhj-ras/spark34/1.txt | 6 +- .../v1-bhj-ras/spark34/10.txt | 24 ++--- .../v1-bhj-ras/spark34/11.txt | 24 ++--- .../v1-bhj-ras/spark34/12.txt | 12 +-- .../v1-bhj-ras/spark34/13.txt | 6 +- .../v1-bhj-ras/spark34/14.txt | 12 +-- .../v1-bhj-ras/spark34/15.txt | 18 ++-- .../v1-bhj-ras/spark34/16.txt | 12 +-- .../v1-bhj-ras/spark34/18.txt | 18 ++-- .../v1-bhj-ras/spark34/19.txt | 12 +-- .../v1-bhj-ras/spark34/20.txt | 30 +++---- .../v1-bhj-ras/spark34/21.txt | 30 +++---- .../v1-bhj-ras/spark34/22.txt | 37 ++++++-- .../v1-bhj-ras/spark34/3.txt | 18 ++-- .../v1-bhj-ras/spark34/4.txt | 12 +-- .../v1-bhj-ras/spark34/5.txt | 36 ++++---- .../v1-bhj-ras/spark34/6.txt | 6 +- .../v1-bhj-ras/spark34/7.txt | 30 +++---- .../v1-bhj-ras/spark34/8.txt | 48 +++++----- .../v1-bhj-ras/spark34/9.txt | 36 ++++---- .../tpch-approved-plan/v1-ras/spark32/1.txt | 6 +- .../tpch-approved-plan/v1-ras/spark32/10.txt | 24 ++--- .../tpch-approved-plan/v1-ras/spark32/11.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark32/12.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark32/13.txt | 6 +- .../tpch-approved-plan/v1-ras/spark32/14.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark32/15.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark32/16.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark32/17.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark32/18.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark32/19.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark32/20.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark32/21.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark32/22.txt | 6 +- .../tpch-approved-plan/v1-ras/spark32/3.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark32/4.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark32/5.txt | 36 ++++---- .../tpch-approved-plan/v1-ras/spark32/6.txt | 6 +- .../tpch-approved-plan/v1-ras/spark32/7.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark32/8.txt | 48 +++++----- .../tpch-approved-plan/v1-ras/spark32/9.txt | 36 ++++---- .../tpch-approved-plan/v1-ras/spark33/1.txt | 6 +- .../tpch-approved-plan/v1-ras/spark33/10.txt | 24 ++--- .../tpch-approved-plan/v1-ras/spark33/11.txt | 24 ++--- .../tpch-approved-plan/v1-ras/spark33/12.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark33/13.txt | 6 +- .../tpch-approved-plan/v1-ras/spark33/14.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark33/15.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark33/16.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark33/17.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark33/18.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark33/19.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark33/20.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark33/21.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark33/22.txt | 37 ++++++-- .../tpch-approved-plan/v1-ras/spark33/3.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark33/4.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark33/5.txt | 36 ++++---- .../tpch-approved-plan/v1-ras/spark33/6.txt | 6 +- .../tpch-approved-plan/v1-ras/spark33/7.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark33/8.txt | 48 +++++----- .../tpch-approved-plan/v1-ras/spark33/9.txt | 36 ++++---- .../tpch-approved-plan/v1-ras/spark34/1.txt | 6 +- .../tpch-approved-plan/v1-ras/spark34/10.txt | 24 ++--- .../tpch-approved-plan/v1-ras/spark34/11.txt | 24 ++--- .../tpch-approved-plan/v1-ras/spark34/12.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark34/13.txt | 6 +- .../tpch-approved-plan/v1-ras/spark34/14.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark34/15.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark34/16.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark34/17.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark34/18.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark34/19.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark34/20.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark34/21.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark34/22.txt | 37 ++++++-- .../tpch-approved-plan/v1-ras/spark34/3.txt | 18 ++-- .../tpch-approved-plan/v1-ras/spark34/4.txt | 12 +-- .../tpch-approved-plan/v1-ras/spark34/5.txt | 36 ++++---- .../tpch-approved-plan/v1-ras/spark34/6.txt | 6 +- .../tpch-approved-plan/v1-ras/spark34/7.txt | 30 +++---- .../tpch-approved-plan/v1-ras/spark34/8.txt | 48 +++++----- .../tpch-approved-plan/v1-ras/spark34/9.txt | 36 ++++---- .../gluten/backendsapi/SparkPlanExecApi.scala | 6 +- .../BasicPhysicalOperatorTransformer.scala | 24 ----- .../columnar/MiscColumnarRules.scala | 2 +- .../columnar/OffloadSingleNode.scala | 49 +---------- .../columnar/PushDownFilterToScan.scala | 66 ++++++++++++++ .../enumerated/EnumeratedTransform.scala | 4 +- .../enumerated/PushFilterToScan.scala | 87 ------------------- .../columnar/enumerated/RemoveFilter.scala | 86 ------------------ .../gluten/planner/cost/LegacyCostModel.scala | 4 - .../gluten/planner/cost/RoughCostModel.scala | 4 - 136 files changed, 1386 insertions(+), 1481 deletions(-) create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PushDownFilterToScan.scala delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/PushFilterToScan.scala delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index d5e079508159..8f7ac330cba5 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -77,6 +77,7 @@ private object CHRuleApi { injector.injectTransform(_ => TransformPreOverrides()) injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectTransform(c => RewriteTransformer.apply(c.session)) + injector.injectTransform(_ => PushDownFilterToScan) injector.injectTransform(_ => PushDownInputFileExpression.PostOffload) injector.injectTransform(_ => EnsureLocalSortRequirements) injector.injectTransform(_ => EliminateLocalSort) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index 48ab172a42e8..903523791a1b 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -842,7 +842,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { // Let's make push down functionally same as vanilla Spark for now. sparkExecNode match { - case fileSourceScan: FileSourceScanExec + case fileSourceScan: FileSourceScanExecTransformerBase if isParquetFormat(fileSourceScan.relation.fileFormat) => PushDownUtil.removeNotSupportPushDownFilters( fileSourceScan.conf, diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 7f95c78bed64..ffbb393bef17 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -68,6 +68,7 @@ private object VeloxRuleApi { injector.injectTransform(_ => TransformPreOverrides()) injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectTransform(c => RewriteTransformer.apply(c.session)) + injector.injectTransform(_ => PushDownFilterToScan) injector.injectTransform(_ => PushDownInputFileExpression.PostOffload) injector.injectTransform(_ => EnsureLocalSortRequirements) injector.injectTransform(_ => EliminateLocalSort) @@ -110,6 +111,7 @@ private object VeloxRuleApi { injector.inject(_ => RemoveTransitions) injector.inject(_ => RemoveNativeWriteFilesSortAndProject()) injector.inject(c => RewriteTransformer.apply(c.session)) + injector.inject(_ => PushDownFilterToScan) injector.inject(_ => PushDownInputFileExpression.PostOffload) injector.inject(_ => EnsureLocalSortRequirements) injector.inject(_ => EliminateLocalSort) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt index 37622c9e13a7..b1b2a7507c21 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (30) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == Sort (29) @@ -35,9 +35,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-02)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] +Arguments: (isnotnull(l_shipdate#X) AND (l_shipdate#X <= 1998-09-02)) (3) ProjectExecTransformer Output [8]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) AS _pre_X#X, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true)) * promote_precision(cast(CheckOverflow((1.00 + promote_precision(cast(l_tax#X as decimal(13,2)))), DecimalType(13,2), true) as decimal(26,4)))), DecimalType(38,6), true) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt index 10c654689b56..a87354004bf3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt @@ -17,24 +17,24 @@ AdaptiveSparkPlan (68) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (21) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) - : : :- ^ NoopFilter (2) + : : :- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) - : : +- ^ NoopFilter (4) + : : +- ^ FilterExecTransformer (4) : : +- ^ Scan parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) - : +- ^ NoopFilter (14) + : +- ^ FilterExecTransformer (14) : +- ^ Scan parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27) +- ColumnarBroadcastExchange (26) - +- ^ NoopFilter (24) + +- ^ FilterExecTransformer (24) +- ^ Scan parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) @@ -69,9 +69,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -80,9 +80,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-10-01), LessThan(o_orderdate,1994-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-10-01)) AND (o_orderdate#X < 1994-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (5) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] @@ -122,9 +122,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_returnflag), EqualTo(l_returnflag,R), IsNotNull(l_orderkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] +Arguments: ((isnotnull(l_returnflag#X) AND (l_returnflag#X = R)) AND isnotnull(l_orderkey#X)) (15) ProjectExecTransformer Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -164,9 +164,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(24) NoopFilter +(24) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (25) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt index 8289ecd8da6c..5ad408607111 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt @@ -19,18 +19,18 @@ AdaptiveSparkPlan (60) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (20) :- ^ ProjectExecTransformer (11) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) - : :- ^ NoopFilter (2) + : :- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (9) : +- BroadcastQueryStage (7) : +- ColumnarBroadcastExchange (6) - : +- ^ NoopFilter (4) + : +- ^ FilterExecTransformer (4) : +- ^ Scan parquet (3) +- ^ InputIteratorTransformer (19) +- BroadcastQueryStage (17) +- ColumnarBroadcastExchange (16) +- ^ ProjectExecTransformer (14) - +- ^ NoopFilter (13) + +- ^ FilterExecTransformer (13) +- ^ Scan parquet (12) +- == Initial Plan == Sort (59) @@ -61,9 +61,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (3) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] @@ -72,9 +72,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (5) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -110,9 +110,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(13) NoopFilter +(13) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) (14) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt index cb6c195e3554..0bbd522e9ca7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt @@ -19,10 +19,10 @@ AdaptiveSparkPlan (44) :- ^ InputIteratorTransformer (7) : +- BroadcastQueryStage (5) : +- ColumnarBroadcastExchange (4) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ ProjectExecTransformer (10) - +- ^ NoopFilter (9) + +- ^ FilterExecTransformer (9) +- ^ Scan parquet (8) +- == Initial Plan == Sort (43) @@ -47,9 +47,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderpriority#X] +Arguments: isnotnull(o_orderkey#X) (3) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_orderpriority#X] @@ -76,9 +76,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_shipdate), In(l_shipmode, [MAIL,SHIP]), GreaterThanOrEqual(l_receiptdate,1994-01-01), LessThan(l_receiptdate,1995-01-01), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] -Arguments: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] +Arguments: ((((((((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND isnotnull(l_shipdate#X)) AND l_shipmode#X IN (MAIL,SHIP)) AND (l_commitdate#X < l_receiptdate#X)) AND (l_shipdate#X < l_commitdate#X)) AND (l_receiptdate#X >= 1994-01-01)) AND (l_receiptdate#X < 1995-01-01)) AND isnotnull(l_orderkey#X)) (10) ProjectExecTransformer Output [2]: [l_orderkey#X, l_shipmode#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt index 2af2ba838800..eceeb7738e8b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (55) +- BroadcastQueryStage (7) +- ColumnarBroadcastExchange (6) +- ^ ProjectExecTransformer (4) - +- ^ NoopFilter (3) + +- ^ FilterExecTransformer (3) +- ^ Scan parquet (2) +- == Initial Plan == Sort (54) @@ -62,9 +62,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(3) NoopFilter +(3) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -Arguments: [o_orderkey#X, o_custkey#X, o_comment#X] +Arguments: ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) (4) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt index 4a18ef6fee83..f991f7b32cb0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt @@ -12,12 +12,12 @@ AdaptiveSparkPlan (35) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8) +- ColumnarBroadcastExchange (7) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == HashAggregate (34) @@ -40,9 +40,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) (3) ProjectExecTransformer Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] @@ -55,9 +55,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: isnotnull(p_partkey#X) (6) WholeStageCodegenTransformer (X) Input [2]: [p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt index ea207bd018d6..c5c3670b3108 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (46) :- ^ InputIteratorTransformer (7) : +- BroadcastQueryStage (5) : +- ColumnarBroadcastExchange (4) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (46) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) - +- ^ NoopFilter (9) + +- ^ FilterExecTransformer (9) +- ^ Scan parquet (8) +- == Initial Plan == Sort (45) @@ -49,9 +49,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] +Arguments: isnotnull(s_suppkey#X) (3) WholeStageCodegenTransformer (X) Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -78,9 +78,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01), IsNotNull(l_suppkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) AND isnotnull(l_suppkey#X)) (10) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt index 8b84f9db6599..51e5f9c64466 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt @@ -23,12 +23,12 @@ AdaptiveSparkPlan (59) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) - :- ^ NoopFilter (2) + :- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7) +- ColumnarBroadcastExchange (6) - +- ^ NoopFilter (4) + +- ^ FilterExecTransformer (4) +- ^ Scan parquet (3) +- == Initial Plan == Sort (58) @@ -60,9 +60,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -Arguments: [ps_partkey#X, ps_suppkey#X] +Arguments: isnotnull(ps_partkey#X) (3) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -71,9 +71,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_type), Not(EqualTo(p_brand,Brand#X)), Not(StringStartsWith(p_type,MEDIUM POLISHED)), In(p_size, [14,19,23,3,36,45,49,9]), IsNotNull(p_partkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: [p_partkey#X, p_brand#X, p_type#X, p_size#X] +Arguments: (((((isnotnull(p_brand#X) AND isnotnull(p_type#X)) AND NOT (p_brand#X = Brand#X)) AND NOT StartsWith(p_type#X, MEDIUM POLISHED)) AND p_size#X IN (49,14,23,45,19,3,36,9)) AND isnotnull(p_partkey#X)) (5) WholeStageCodegenTransformer (X) Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt index 87714404df3e..d6528a10e0d6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt @@ -17,10 +17,10 @@ AdaptiveSparkPlan (88) : :- ^ InputIteratorTransformer (7) : : +- BroadcastQueryStage (5) : : +- ColumnarBroadcastExchange (4) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) - : :- ^ NoopFilter (9) + : :- ^ FilterExecTransformer (9) : : +- ^ Scan parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (88) +- BroadcastQueryStage (39) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) - :- ^ NoopFilter (31) + :- ^ FilterExecTransformer (31) : +- ^ Scan parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33) @@ -86,9 +86,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_name#X] -Arguments: [c_custkey#X, c_name#X] +Arguments: isnotnull(c_custkey#X) (3) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_name#X] @@ -115,9 +115,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] +Arguments: (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) (10) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] @@ -212,9 +212,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(31) NoopFilter +(31) FilterExecTransformer Input [2]: [l_orderkey#X, l_quantity#X] -Arguments: [l_orderkey#X, l_quantity#X] +Arguments: isnotnull(l_orderkey#X) (32) ReusedExchange [Reuses operator id: 23] Output [1]: [l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt index 6ef3621c3a50..218c7e7d3682 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt @@ -11,12 +11,12 @@ AdaptiveSparkPlan (34) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8) +- ColumnarBroadcastExchange (7) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == HashAggregate (33) @@ -39,9 +39,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] +Arguments: ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) (3) ProjectExecTransformer Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -54,9 +54,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_size#X, p_container#X] +Arguments: (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) (6) WholeStageCodegenTransformer (X) Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt index ee9964166168..925c9b4c1df7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt @@ -17,7 +17,7 @@ AdaptiveSparkPlan (112) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (52) : +- BroadcastQueryStage (50) @@ -28,13 +28,13 @@ AdaptiveSparkPlan (112) : : +- BroadcastQueryStage (24) : : +- ColumnarBroadcastExchange (23) : : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (21) - : : :- ^ NoopFilter (12) + : : :- ^ FilterExecTransformer (12) : : : +- ^ Scan parquet (11) : : +- ^ InputIteratorTransformer (20) : : +- BroadcastQueryStage (18) : : +- ColumnarBroadcastExchange (17) : : +- ^ ProjectExecTransformer (15) - : : +- ^ NoopFilter (14) + : : +- ^ FilterExecTransformer (14) : : +- ^ Scan parquet (13) : +- ^ FilterExecTransformer (45) : +- ^ ProjectExecTransformer (44) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (112) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) : :- ^ ProjectExecTransformer (29) - : : +- ^ NoopFilter (28) + : : +- ^ FilterExecTransformer (28) : : +- ^ Scan parquet (27) : +- ^ InputIteratorTransformer (33) : +- BroadcastQueryStage (31) @@ -56,7 +56,7 @@ AdaptiveSparkPlan (112) +- BroadcastQueryStage (60) +- ColumnarBroadcastExchange (59) +- ^ ProjectExecTransformer (57) - +- ^ NoopFilter (56) + +- ^ FilterExecTransformer (56) +- ^ Scan parquet (55) +- == Initial Plan == Sort (111) @@ -106,9 +106,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] +Arguments: isnotnull(s_nationkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -147,9 +147,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(12) NoopFilter +(12) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] +Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) (13) Scan parquet Output [2]: [p_partkey#X, p_name#X] @@ -158,9 +158,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) (15) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -214,9 +214,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -Arguments: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) (29) ProjectExecTransformer Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] @@ -331,9 +331,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(56) NoopFilter +(56) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) (57) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt index 4da38d933ee8..0ef77a3b4058 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt @@ -19,12 +19,12 @@ AdaptiveSparkPlan (93) : : :- ^ InputIteratorTransformer (7) : : : +- BroadcastQueryStage (5) : : : +- ColumnarBroadcastExchange (4) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14) @@ -34,19 +34,19 @@ AdaptiveSparkPlan (93) : : +- BroadcastQueryStage (23) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) - : : +- ^ NoopFilter (19) + : : +- ^ FilterExecTransformer (19) : : +- ^ Scan parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) - : +- ^ NoopFilter (30) + : +- ^ FilterExecTransformer (30) : +- ^ Scan parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) - +- ^ NoopFilter (40) + +- ^ FilterExecTransformer (40) +- ^ Scan parquet (39) +- == Initial Plan == TakeOrderedAndProject (92) @@ -90,9 +90,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] @@ -119,9 +119,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (10) ProjectExecTransformer Output [2]: [l_orderkey#X, l_suppkey#X] @@ -163,9 +163,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] ReadSchema: struct -(19) NoopFilter +(19) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) (20) ProjectExecTransformer Output [2]: [l_orderkey#X, l_suppkey#X] @@ -210,9 +210,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(30) NoopFilter +(30) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderstatus#X] -Arguments: [o_orderkey#X, o_orderstatus#X] +Arguments: ((isnotnull(o_orderstatus#X) AND (o_orderstatus#X = F)) AND isnotnull(o_orderkey#X)) (31) ProjectExecTransformer Output [1]: [o_orderkey#X] @@ -252,9 +252,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(40) NoopFilter +(40) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = SAUDI ARABIA)) AND isnotnull(n_nationkey#X)) (41) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt index cfe29976b36a..e3cda8bcfc8b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (40) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) - :- ^ NoopFilter (2) + :- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6) @@ -43,9 +43,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: [c_custkey#X, c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) (3) Scan parquet Output [1]: [o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt index eacdb0cccc59..4e3cdc99706b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt @@ -19,15 +19,15 @@ AdaptiveSparkPlan (54) : : +- BroadcastQueryStage (6) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) - : +- ^ NoopFilter (10) + : +- ^ FilterExecTransformer (10) : +- ^ Scan parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) - +- ^ NoopFilter (14) + +- ^ FilterExecTransformer (14) +- ^ Scan parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) @@ -57,9 +57,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_mktsegment#X] -Arguments: [c_custkey#X, c_mktsegment#X] +Arguments: ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) (3) ProjectExecTransformer Output [1]: [c_custkey#X] @@ -90,9 +90,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] +Arguments: (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [c_custkey#X] @@ -110,9 +110,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) (15) ProjectExecTransformer Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt index afc352febbc3..1332c8ba9663 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt @@ -17,13 +17,13 @@ AdaptiveSparkPlan (46) +- ^ ProjectExecTransformer (13) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == Sort (45) @@ -49,9 +49,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] +Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_orderdate#X < 1993-10-01)) (3) ProjectExecTransformer Output [2]: [o_orderkey#X, o_orderpriority#X] @@ -64,9 +64,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND (l_commitdate#X < l_receiptdate#X)) (6) ProjectExecTransformer Output [1]: [l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt index de7994a61b14..1445f75cfcb4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt @@ -27,31 +27,31 @@ AdaptiveSparkPlan (102) : : : : :- ^ InputIteratorTransformer (7) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ ProjectExecTransformer (10) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) - : : : +- ^ NoopFilter (14) + : : : +- ^ FilterExecTransformer (14) : : : +- ^ Scan parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) - : : +- ^ NoopFilter (23) + : : +- ^ FilterExecTransformer (23) : : +- ^ Scan parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) - : +- ^ NoopFilter (32) + : +- ^ FilterExecTransformer (32) : +- ^ Scan parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) - +- ^ NoopFilter (41) + +- ^ FilterExecTransformer (41) +- ^ Scan parquet (40) +- == Initial Plan == Sort (101) @@ -97,9 +97,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -126,9 +126,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1994-01-01)) AND (o_orderdate#X < 1995-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (10) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] @@ -150,9 +150,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: (isnotnull(l_orderkey#X) AND isnotnull(l_suppkey#X)) (15) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -188,9 +188,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (24) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -226,9 +226,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_name#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (33) WholeStageCodegenTransformer (X) Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] @@ -264,9 +264,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,ASIA), IsNotNull(r_regionkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = ASIA)) AND isnotnull(r_regionkey#X)) (42) ProjectExecTransformer Output [1]: [r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt index 451b7abd33b4..2b97fd28f147 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt @@ -9,7 +9,7 @@ AdaptiveSparkPlan (20) +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == HashAggregate (19) @@ -27,9 +27,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), IsNotNull(l_discount), IsNotNull(l_quantity), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), GreaterThanOrEqual(l_discount,0.05), LessThanOrEqual(l_discount,0.07), LessThan(l_quantity,24.00)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((((((isnotnull(l_shipdate#X) AND isnotnull(l_discount#X)) AND isnotnull(l_quantity#X)) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND (l_discount#X >= 0.05)) AND (l_discount#X <= 0.07)) AND (l_quantity#X < 24.00)) (3) ProjectExecTransformer Output [3]: [l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(l_extendedprice#X) * promote_precision(l_discount#X)), DecimalType(25,4), true) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt index 04e9c12e2483..5f3cf3fa7c56 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt @@ -27,24 +27,24 @@ AdaptiveSparkPlan (95) : : : : :- ^ InputIteratorTransformer (7) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16) : : : +- ColumnarBroadcastExchange (15) - : : : +- ^ NoopFilter (13) + : : : +- ^ FilterExecTransformer (13) : : : +- ^ Scan parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25) : : +- ColumnarBroadcastExchange (24) - : : +- ^ NoopFilter (22) + : : +- ^ FilterExecTransformer (22) : : +- ^ Scan parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) - : +- ^ NoopFilter (31) + : +- ^ FilterExecTransformer (31) : +- ^ Scan parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40) @@ -91,9 +91,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -120,9 +120,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-01-01), LessThanOrEqual(l_shipdate,1996-12-31), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-01-01)) AND (l_shipdate#X <= 1996-12-31)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (10) BroadcastHashJoinExecTransformer Left keys [1]: [s_suppkey#X] @@ -140,9 +140,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(13) NoopFilter +(13) FilterExecTransformer Input [2]: [o_orderkey#X, o_custkey#X] -Arguments: [o_orderkey#X, o_custkey#X] +Arguments: (isnotnull(o_orderkey#X) AND isnotnull(o_custkey#X)) (14) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_custkey#X] @@ -178,9 +178,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(22) NoopFilter +(22) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (23) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -216,9 +216,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), Or(EqualTo(n_name,FRANCE),EqualTo(n_name,GERMANY))] ReadSchema: struct -(31) NoopFilter +(31) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: (isnotnull(n_nationkey#X) AND ((n_name#X = FRANCE) OR (n_name#X = GERMANY))) (32) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt index e8e36d54cc4a..a64588c5ba98 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt @@ -33,40 +33,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- BroadcastQueryStage (6) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) - : : : : : : : +- ^ NoopFilter (2) + : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) - : : : : : : +- ^ NoopFilter (10) + : : : : : : +- ^ FilterExecTransformer (10) : : : : : : +- ^ Scan parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17) : : : : : +- ColumnarBroadcastExchange (16) - : : : : : +- ^ NoopFilter (14) + : : : : : +- ^ FilterExecTransformer (14) : : : : : +- ^ Scan parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26) : : : : +- ColumnarBroadcastExchange (25) - : : : : +- ^ NoopFilter (23) + : : : : +- ^ FilterExecTransformer (23) : : : : +- ^ Scan parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35) : : : +- ColumnarBroadcastExchange (34) - : : : +- ^ NoopFilter (32) + : : : +- ^ FilterExecTransformer (32) : : : +- ^ Scan parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44) : : +- ColumnarBroadcastExchange (43) - : : +- ^ NoopFilter (41) + : : +- ^ FilterExecTransformer (41) : : +- ^ Scan parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53) : +- ColumnarBroadcastExchange (52) - : +- ^ NoopFilter (50) + : +- ^ FilterExecTransformer (50) : +- ^ Scan parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) - +- ^ NoopFilter (59) + +- ^ FilterExecTransformer (59) +- ^ Scan parquet (58) +- == Initial Plan == Sort (130) @@ -122,9 +122,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_type), EqualTo(p_type,ECONOMY ANODIZED STEEL), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: ((isnotnull(p_type#X) AND (p_type#X = ECONOMY ANODIZED STEEL)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -155,9 +155,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [p_partkey#X] @@ -175,9 +175,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (15) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -213,9 +213,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1995-01-01), LessThanOrEqual(o_orderdate,1996-12-31), IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1995-01-01)) AND (o_orderdate#X <= 1996-12-31)) AND isnotnull(o_orderkey#X)) AND isnotnull(o_custkey#X)) (24) WholeStageCodegenTransformer (X) Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -251,9 +251,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (33) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -289,9 +289,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [n_nationkey#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (42) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_regionkey#X] @@ -327,9 +327,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(50) NoopFilter +(50) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (51) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] @@ -365,9 +365,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AMERICA), IsNotNull(r_regionkey)] ReadSchema: struct -(59) NoopFilter +(59) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = AMERICA)) AND isnotnull(r_regionkey#X)) (60) ProjectExecTransformer Output [1]: [r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt index a91e180bd632..45bfb4c1996d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt @@ -28,29 +28,29 @@ AdaptiveSparkPlan (100) : : : : : +- BroadcastQueryStage (6) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) - : : : : +- ^ NoopFilter (10) + : : : : +- ^ FilterExecTransformer (10) : : : : +- ^ Scan parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) - : : : +- ^ NoopFilter (14) + : : : +- ^ FilterExecTransformer (14) : : : +- ^ Scan parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) - : : +- ^ NoopFilter (23) + : : +- ^ FilterExecTransformer (23) : : +- ^ Scan parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) - : +- ^ NoopFilter (32) + : +- ^ FilterExecTransformer (32) : +- ^ Scan parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) - +- ^ NoopFilter (41) + +- ^ FilterExecTransformer (41) +- ^ Scan parquet (40) +- == Initial Plan == Sort (99) @@ -95,9 +95,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringContains(p_name,green), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: ((isnotnull(p_name#X) AND Contains(p_name#X, green)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -128,9 +128,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [p_partkey#X] @@ -148,9 +148,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (15) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -186,9 +186,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] +Arguments: (isnotnull(ps_suppkey#X) AND isnotnull(ps_partkey#X)) (24) WholeStageCodegenTransformer (X) Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -224,9 +224,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_orderdate#X] +Arguments: isnotnull(o_orderkey#X) (33) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_orderdate#X] @@ -262,9 +262,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (42) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt index 7e1dc79b38aa..417ddf9de25d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (30) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == Sort (29) @@ -35,9 +35,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-02)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] +Arguments: (isnotnull(l_shipdate#X) AND (l_shipdate#X <= 1998-09-02)) (3) ProjectExecTransformer Output [8]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) AS _pre_X#X, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4))) * promote_precision(cast(CheckOverflow((1.00 + promote_precision(cast(l_tax#X as decimal(13,2)))), DecimalType(13,2)) as decimal(26,4)))), DecimalType(38,6)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt index 8b28d73f9f04..7b8d0238637d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt @@ -17,24 +17,24 @@ AdaptiveSparkPlan (68) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (21) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) - : : :- ^ NoopFilter (2) + : : :- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) - : : +- ^ NoopFilter (4) + : : +- ^ FilterExecTransformer (4) : : +- ^ Scan parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) - : +- ^ NoopFilter (14) + : +- ^ FilterExecTransformer (14) : +- ^ Scan parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) - +- ^ NoopFilter (24) + +- ^ FilterExecTransformer (24) +- ^ Scan parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) @@ -69,9 +69,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -80,9 +80,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-10-01), LessThan(o_orderdate,1994-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-10-01)) AND (o_orderdate#X < 1994-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (5) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] @@ -122,9 +122,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_returnflag), EqualTo(l_returnflag,R), IsNotNull(l_orderkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] +Arguments: ((isnotnull(l_returnflag#X) AND (l_returnflag#X = R)) AND isnotnull(l_orderkey#X)) (15) ProjectExecTransformer Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -164,9 +164,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(24) NoopFilter +(24) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (25) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt index 0b96c3470133..e01bfff2886b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt @@ -19,18 +19,18 @@ AdaptiveSparkPlan (60) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (20) :- ^ ProjectExecTransformer (11) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) - : :- ^ NoopFilter (2) + : :- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (9) : +- BroadcastQueryStage (7), Statistics(X) : +- ColumnarBroadcastExchange (6) - : +- ^ NoopFilter (4) + : +- ^ FilterExecTransformer (4) : +- ^ Scan parquet (3) +- ^ InputIteratorTransformer (19) +- BroadcastQueryStage (17), Statistics(X) +- ColumnarBroadcastExchange (16) +- ^ ProjectExecTransformer (14) - +- ^ NoopFilter (13) + +- ^ FilterExecTransformer (13) +- ^ Scan parquet (12) +- == Initial Plan == Sort (59) @@ -61,9 +61,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (3) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] @@ -72,9 +72,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (5) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -110,9 +110,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(13) NoopFilter +(13) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) (14) ProjectExecTransformer Output [1]: [n_nationkey#X] @@ -336,7 +336,7 @@ AdaptiveSparkPlan (102) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) :- ^ ProjectExecTransformer (68) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (67) - : :- ^ NoopFilter (62) + : :- ^ FilterExecTransformer (62) : : +- ^ Scan parquet (61) : +- ^ InputIteratorTransformer (66) : +- BroadcastQueryStage (64), Statistics(X) @@ -370,9 +370,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (63) ReusedExchange [Reuses operator id: 6] Output [2]: [s_suppkey#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt index 663f4d6d8073..490806d71daa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt @@ -19,10 +19,10 @@ AdaptiveSparkPlan (44) :- ^ InputIteratorTransformer (7) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ ProjectExecTransformer (10) - +- ^ NoopFilter (9) + +- ^ FilterExecTransformer (9) +- ^ Scan parquet (8) +- == Initial Plan == Sort (43) @@ -47,9 +47,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderpriority#X] +Arguments: isnotnull(o_orderkey#X) (3) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_orderpriority#X] @@ -76,9 +76,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_shipdate), In(l_shipmode, [MAIL,SHIP]), GreaterThanOrEqual(l_receiptdate,1994-01-01), LessThan(l_receiptdate,1995-01-01), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] -Arguments: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] +Arguments: ((((((((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND isnotnull(l_shipdate#X)) AND l_shipmode#X IN (MAIL,SHIP)) AND (l_commitdate#X < l_receiptdate#X)) AND (l_shipdate#X < l_commitdate#X)) AND (l_receiptdate#X >= 1994-01-01)) AND (l_receiptdate#X < 1995-01-01)) AND isnotnull(l_orderkey#X)) (10) ProjectExecTransformer Output [2]: [l_orderkey#X, l_shipmode#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt index a51e79a7f4f5..c3ac3e0903c8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (55) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ ProjectExecTransformer (4) - +- ^ NoopFilter (3) + +- ^ FilterExecTransformer (3) +- ^ Scan parquet (2) +- == Initial Plan == Sort (54) @@ -62,9 +62,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(3) NoopFilter +(3) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -Arguments: [o_orderkey#X, o_custkey#X, o_comment#X] +Arguments: ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) (4) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt index d349e169adf7..67bce21483fb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt @@ -12,12 +12,12 @@ AdaptiveSparkPlan (35) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == HashAggregate (34) @@ -40,9 +40,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) (3) ProjectExecTransformer Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] @@ -55,9 +55,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: isnotnull(p_partkey#X) (6) WholeStageCodegenTransformer (X) Input [2]: [p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt index 632022609475..9a7879d5c1d1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt @@ -11,7 +11,7 @@ AdaptiveSparkPlan (43) :- ^ InputIteratorTransformer (7) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) @@ -22,7 +22,7 @@ AdaptiveSparkPlan (43) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) - +- ^ NoopFilter (9) + +- ^ FilterExecTransformer (9) +- ^ Scan parquet (8) +- == Initial Plan == Sort (42) @@ -48,9 +48,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] +Arguments: isnotnull(s_suppkey#X) (3) WholeStageCodegenTransformer (X) Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -77,9 +77,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01), IsNotNull(l_suppkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) AND isnotnull(l_suppkey#X)) (10) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) AS _pre_X#X] @@ -251,7 +251,7 @@ AdaptiveSparkPlan (69) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) - +- ^ NoopFilter (45) + +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) +- == Initial Plan == HashAggregate (68) @@ -271,9 +271,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) (46) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt index 0dd714b70098..6faaec51bb6b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt @@ -23,12 +23,12 @@ AdaptiveSparkPlan (59) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) - :- ^ NoopFilter (2) + :- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) - +- ^ NoopFilter (4) + +- ^ FilterExecTransformer (4) +- ^ Scan parquet (3) +- == Initial Plan == Sort (58) @@ -60,9 +60,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -Arguments: [ps_partkey#X, ps_suppkey#X] +Arguments: isnotnull(ps_partkey#X) (3) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -71,9 +71,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_type), Not(EqualTo(p_brand,Brand#X)), Not(StringStartsWith(p_type,MEDIUM POLISHED)), In(p_size, [14,19,23,3,36,45,49,9]), IsNotNull(p_partkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: [p_partkey#X, p_brand#X, p_type#X, p_size#X] +Arguments: (((((isnotnull(p_brand#X) AND isnotnull(p_type#X)) AND NOT (p_brand#X = Brand#X)) AND NOT StartsWith(p_type#X, MEDIUM POLISHED)) AND p_size#X IN (49,14,23,45,19,3,36,9)) AND isnotnull(p_partkey#X)) (5) WholeStageCodegenTransformer (X) Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt index ea324851532c..661a04a3db6a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt @@ -17,10 +17,10 @@ AdaptiveSparkPlan (88) : :- ^ InputIteratorTransformer (7) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) - : :- ^ NoopFilter (9) + : :- ^ FilterExecTransformer (9) : : +- ^ Scan parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (88) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) - :- ^ NoopFilter (31) + :- ^ FilterExecTransformer (31) : +- ^ Scan parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) @@ -86,9 +86,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_name#X] -Arguments: [c_custkey#X, c_name#X] +Arguments: isnotnull(c_custkey#X) (3) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_name#X] @@ -115,9 +115,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] +Arguments: (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) (10) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] @@ -212,9 +212,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(31) NoopFilter +(31) FilterExecTransformer Input [2]: [l_orderkey#X, l_quantity#X] -Arguments: [l_orderkey#X, l_quantity#X] +Arguments: isnotnull(l_orderkey#X) (32) ReusedExchange [Reuses operator id: 23] Output [1]: [l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt index fa391e426cd0..47e5d26dbdd6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt @@ -11,12 +11,12 @@ AdaptiveSparkPlan (34) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == HashAggregate (33) @@ -39,9 +39,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] +Arguments: ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) (3) ProjectExecTransformer Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -54,9 +54,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_size#X, p_container#X] +Arguments: (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) (6) WholeStageCodegenTransformer (X) Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt index 9bfc87c2bd22..b15eb6d9c365 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (109) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (52) : +- BroadcastQueryStage (50), Statistics(X) @@ -27,13 +27,13 @@ AdaptiveSparkPlan (109) : : +- BroadcastQueryStage (24), Statistics(X) : : +- ColumnarBroadcastExchange (23) : : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (21) - : : :- ^ NoopFilter (12) + : : :- ^ FilterExecTransformer (12) : : : +- ^ Scan parquet (11) : : +- ^ InputIteratorTransformer (20) : : +- BroadcastQueryStage (18), Statistics(X) : : +- ColumnarBroadcastExchange (17) : : +- ^ ProjectExecTransformer (15) - : : +- ^ NoopFilter (14) + : : +- ^ FilterExecTransformer (14) : : +- ^ Scan parquet (13) : +- ^ FilterExecTransformer (45) : +- ^ ProjectExecTransformer (44) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (109) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) : :- ^ ProjectExecTransformer (29) - : : +- ^ NoopFilter (28) + : : +- ^ FilterExecTransformer (28) : : +- ^ Scan parquet (27) : +- ^ InputIteratorTransformer (33) : +- BroadcastQueryStage (31), Statistics(X) @@ -55,7 +55,7 @@ AdaptiveSparkPlan (109) +- BroadcastQueryStage (60), Statistics(X) +- ColumnarBroadcastExchange (59) +- ^ ProjectExecTransformer (57) - +- ^ NoopFilter (56) + +- ^ FilterExecTransformer (56) +- ^ Scan parquet (55) +- == Initial Plan == Sort (108) @@ -105,9 +105,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] +Arguments: isnotnull(s_nationkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -146,9 +146,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(12) NoopFilter +(12) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] +Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) (13) Scan parquet Output [2]: [p_partkey#X, p_name#X] @@ -157,9 +157,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) (15) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -213,9 +213,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -Arguments: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) (29) ProjectExecTransformer Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] @@ -330,9 +330,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(56) NoopFilter +(56) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) (57) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt index 0ab2c88fdef9..8a2bf8db611d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt @@ -18,12 +18,12 @@ AdaptiveSparkPlan (92) : : :- ^ InputIteratorTransformer (7) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) @@ -33,19 +33,19 @@ AdaptiveSparkPlan (92) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) - : : +- ^ NoopFilter (19) + : : +- ^ FilterExecTransformer (19) : : +- ^ Scan parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) - : +- ^ NoopFilter (30) + : +- ^ FilterExecTransformer (30) : +- ^ Scan parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) - +- ^ NoopFilter (40) + +- ^ FilterExecTransformer (40) +- ^ Scan parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) @@ -89,9 +89,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] @@ -118,9 +118,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (10) ProjectExecTransformer Output [2]: [l_orderkey#X, l_suppkey#X] @@ -162,9 +162,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] ReadSchema: struct -(19) NoopFilter +(19) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) (20) ProjectExecTransformer Output [2]: [l_orderkey#X, l_suppkey#X] @@ -209,9 +209,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(30) NoopFilter +(30) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderstatus#X] -Arguments: [o_orderkey#X, o_orderstatus#X] +Arguments: ((isnotnull(o_orderstatus#X) AND (o_orderstatus#X = F)) AND isnotnull(o_orderkey#X)) (31) ProjectExecTransformer Output [1]: [o_orderkey#X] @@ -251,9 +251,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(40) NoopFilter +(40) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = SAUDI ARABIA)) AND isnotnull(n_nationkey#X)) (41) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt index 44d408a45c82..e7ea95ab686f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (40) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) - :- ^ NoopFilter (2) + :- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) @@ -43,9 +43,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: [c_custkey#X, c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) (3) Scan parquet Output [1]: [o_custkey#X] @@ -215,7 +215,7 @@ Arguments: isFinalPlan=true ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#X, [id=#X] AdaptiveSparkPlan (60) +- == Final Plan == VeloxColumnarToRowExec (53) @@ -226,7 +226,7 @@ AdaptiveSparkPlan (60) +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) - +- ^ NoopFilter (42) + +- ^ FilterExecTransformer (42) +- ^ Scan parquet (41) +- == Initial Plan == HashAggregate (59) @@ -244,9 +244,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal), GreaterThan(c_acctbal,0.00)] ReadSchema: struct -(42) NoopFilter +(42) FilterExecTransformer Input [2]: [c_phone#X, c_acctbal#X] -Arguments: [c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND (c_acctbal#X > 0.00)) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) (43) ProjectExecTransformer Output [1]: [c_acctbal#X] @@ -330,4 +330,25 @@ Results [1]: [avg(c_acctbal#X)#X AS avg(c_acctbal)#X] (60) AdaptiveSparkPlan Output [1]: [avg(c_acctbal)#X] -Arguments: isFinalPlan=true \ No newline at end of file +Arguments: isFinalPlan=true + +Subquery:2 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +AdaptiveSparkPlan (60) ++- == Final Plan == + VeloxColumnarToRowExec (53) + +- ^ RegularHashAggregateExecTransformer (51) + +- ^ InputIteratorTransformer (50) + +- ShuffleQueryStage (48), Statistics(X) + +- ColumnarExchange (47) + +- VeloxResizeBatches (46) + +- ^ FlushableHashAggregateExecTransformer (44) + +- ^ ProjectExecTransformer (43) + +- ^ FilterExecTransformer (42) + +- ^ Scan parquet (41) ++- == Initial Plan == + HashAggregate (59) + +- Exchange (58) + +- HashAggregate (57) + +- Project (56) + +- Filter (55) + +- Scan parquet (54) \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt index cab0ccc1015a..03ff6cc3d00e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt @@ -19,15 +19,15 @@ AdaptiveSparkPlan (54) : : +- BroadcastQueryStage (6), Statistics(X) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) - : +- ^ NoopFilter (10) + : +- ^ FilterExecTransformer (10) : +- ^ Scan parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) - +- ^ NoopFilter (14) + +- ^ FilterExecTransformer (14) +- ^ Scan parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) @@ -57,9 +57,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_mktsegment#X] -Arguments: [c_custkey#X, c_mktsegment#X] +Arguments: ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) (3) ProjectExecTransformer Output [1]: [c_custkey#X] @@ -90,9 +90,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] +Arguments: (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [c_custkey#X] @@ -110,9 +110,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) (15) ProjectExecTransformer Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt index f4eaf3e8dc4a..c6f0f246e5fe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt @@ -17,13 +17,13 @@ AdaptiveSparkPlan (46) +- ^ ProjectExecTransformer (13) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == Sort (45) @@ -49,9 +49,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] +Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_orderdate#X < 1993-10-01)) (3) ProjectExecTransformer Output [2]: [o_orderkey#X, o_orderpriority#X] @@ -64,9 +64,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND (l_commitdate#X < l_receiptdate#X)) (6) ProjectExecTransformer Output [1]: [l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt index 209fc2b2544c..213c8e698e9d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt @@ -27,31 +27,31 @@ AdaptiveSparkPlan (102) : : : : :- ^ InputIteratorTransformer (7) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ ProjectExecTransformer (10) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) - : : : +- ^ NoopFilter (14) + : : : +- ^ FilterExecTransformer (14) : : : +- ^ Scan parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) - : : +- ^ NoopFilter (23) + : : +- ^ FilterExecTransformer (23) : : +- ^ Scan parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) - : +- ^ NoopFilter (32) + : +- ^ FilterExecTransformer (32) : +- ^ Scan parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) - +- ^ NoopFilter (41) + +- ^ FilterExecTransformer (41) +- ^ Scan parquet (40) +- == Initial Plan == Sort (101) @@ -97,9 +97,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -126,9 +126,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1994-01-01)) AND (o_orderdate#X < 1995-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (10) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] @@ -150,9 +150,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: (isnotnull(l_orderkey#X) AND isnotnull(l_suppkey#X)) (15) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -188,9 +188,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (24) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -226,9 +226,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_name#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (33) WholeStageCodegenTransformer (X) Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] @@ -264,9 +264,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,ASIA), IsNotNull(r_regionkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = ASIA)) AND isnotnull(r_regionkey#X)) (42) ProjectExecTransformer Output [1]: [r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt index ee04d2159020..bb9d04cdcdc7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt @@ -9,7 +9,7 @@ AdaptiveSparkPlan (20) +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == HashAggregate (19) @@ -27,9 +27,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), IsNotNull(l_discount), IsNotNull(l_quantity), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), GreaterThanOrEqual(l_discount,0.05), LessThanOrEqual(l_discount,0.07), LessThan(l_quantity,24.00)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((((((isnotnull(l_shipdate#X) AND isnotnull(l_discount#X)) AND isnotnull(l_quantity#X)) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND (l_discount#X >= 0.05)) AND (l_discount#X <= 0.07)) AND (l_quantity#X < 24.00)) (3) ProjectExecTransformer Output [3]: [l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(l_extendedprice#X) * promote_precision(l_discount#X)), DecimalType(25,4)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt index 109de659cb1f..3f54f20f050e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt @@ -27,24 +27,24 @@ AdaptiveSparkPlan (95) : : : : :- ^ InputIteratorTransformer (7) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) - : : : +- ^ NoopFilter (13) + : : : +- ^ FilterExecTransformer (13) : : : +- ^ Scan parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) - : : +- ^ NoopFilter (22) + : : +- ^ FilterExecTransformer (22) : : +- ^ Scan parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) - : +- ^ NoopFilter (31) + : +- ^ FilterExecTransformer (31) : +- ^ Scan parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) @@ -91,9 +91,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -120,9 +120,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-01-01), LessThanOrEqual(l_shipdate,1996-12-31), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-01-01)) AND (l_shipdate#X <= 1996-12-31)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (10) BroadcastHashJoinExecTransformer Left keys [1]: [s_suppkey#X] @@ -140,9 +140,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(13) NoopFilter +(13) FilterExecTransformer Input [2]: [o_orderkey#X, o_custkey#X] -Arguments: [o_orderkey#X, o_custkey#X] +Arguments: (isnotnull(o_orderkey#X) AND isnotnull(o_custkey#X)) (14) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_custkey#X] @@ -178,9 +178,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(22) NoopFilter +(22) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (23) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -216,9 +216,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), Or(EqualTo(n_name,FRANCE),EqualTo(n_name,GERMANY))] ReadSchema: struct -(31) NoopFilter +(31) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: (isnotnull(n_nationkey#X) AND ((n_name#X = FRANCE) OR (n_name#X = GERMANY))) (32) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt index 709642967edd..ba0d25c60800 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt @@ -33,40 +33,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- BroadcastQueryStage (6), Statistics(X) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) - : : : : : : : +- ^ NoopFilter (2) + : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) - : : : : : : +- ^ NoopFilter (10) + : : : : : : +- ^ FilterExecTransformer (10) : : : : : : +- ^ Scan parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) - : : : : : +- ^ NoopFilter (14) + : : : : : +- ^ FilterExecTransformer (14) : : : : : +- ^ Scan parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) - : : : : +- ^ NoopFilter (23) + : : : : +- ^ FilterExecTransformer (23) : : : : +- ^ Scan parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) - : : : +- ^ NoopFilter (32) + : : : +- ^ FilterExecTransformer (32) : : : +- ^ Scan parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) - : : +- ^ NoopFilter (41) + : : +- ^ FilterExecTransformer (41) : : +- ^ Scan parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) - : +- ^ NoopFilter (50) + : +- ^ FilterExecTransformer (50) : +- ^ Scan parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) - +- ^ NoopFilter (59) + +- ^ FilterExecTransformer (59) +- ^ Scan parquet (58) +- == Initial Plan == Sort (130) @@ -122,9 +122,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_type), EqualTo(p_type,ECONOMY ANODIZED STEEL), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: ((isnotnull(p_type#X) AND (p_type#X = ECONOMY ANODIZED STEEL)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -155,9 +155,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [p_partkey#X] @@ -175,9 +175,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (15) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -213,9 +213,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1995-01-01), LessThanOrEqual(o_orderdate,1996-12-31), IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1995-01-01)) AND (o_orderdate#X <= 1996-12-31)) AND isnotnull(o_orderkey#X)) AND isnotnull(o_custkey#X)) (24) WholeStageCodegenTransformer (X) Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -251,9 +251,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (33) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -289,9 +289,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [n_nationkey#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (42) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_regionkey#X] @@ -327,9 +327,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(50) NoopFilter +(50) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (51) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] @@ -365,9 +365,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AMERICA), IsNotNull(r_regionkey)] ReadSchema: struct -(59) NoopFilter +(59) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = AMERICA)) AND isnotnull(r_regionkey#X)) (60) ProjectExecTransformer Output [1]: [r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt index 3b57f40101e8..d15efa6a6567 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt @@ -28,29 +28,29 @@ AdaptiveSparkPlan (100) : : : : : +- BroadcastQueryStage (6), Statistics(X) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) - : : : : +- ^ NoopFilter (10) + : : : : +- ^ FilterExecTransformer (10) : : : : +- ^ Scan parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) - : : : +- ^ NoopFilter (14) + : : : +- ^ FilterExecTransformer (14) : : : +- ^ Scan parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) - : : +- ^ NoopFilter (23) + : : +- ^ FilterExecTransformer (23) : : +- ^ Scan parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) - : +- ^ NoopFilter (32) + : +- ^ FilterExecTransformer (32) : +- ^ Scan parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) - +- ^ NoopFilter (41) + +- ^ FilterExecTransformer (41) +- ^ Scan parquet (40) +- == Initial Plan == Sort (99) @@ -95,9 +95,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringContains(p_name,green), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: ((isnotnull(p_name#X) AND Contains(p_name#X, green)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -128,9 +128,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [p_partkey#X] @@ -148,9 +148,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (15) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -186,9 +186,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] +Arguments: (isnotnull(ps_suppkey#X) AND isnotnull(ps_partkey#X)) (24) WholeStageCodegenTransformer (X) Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -224,9 +224,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_orderdate#X] +Arguments: isnotnull(o_orderkey#X) (33) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_orderdate#X] @@ -262,9 +262,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (42) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt index 03cab493a6c4..450797d3aefd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (30) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == Sort (29) @@ -35,9 +35,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-02)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] +Arguments: (isnotnull(l_shipdate#X) AND (l_shipdate#X <= 1998-09-02)) (3) ProjectExecTransformer Output [8]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, (l_extendedprice#X * (1 - l_discount#X)) AS _pre_X#X, ((l_extendedprice#X * (1 - l_discount#X)) * (1 + l_tax#X)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt index 81f3b18b6506..7a2404c1d610 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt @@ -17,24 +17,24 @@ AdaptiveSparkPlan (68) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (21) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) - : : :- ^ NoopFilter (2) + : : :- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) - : : +- ^ NoopFilter (4) + : : +- ^ FilterExecTransformer (4) : : +- ^ Scan parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) - : +- ^ NoopFilter (14) + : +- ^ FilterExecTransformer (14) : +- ^ Scan parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) - +- ^ NoopFilter (24) + +- ^ FilterExecTransformer (24) +- ^ Scan parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) @@ -69,9 +69,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -80,9 +80,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-10-01), LessThan(o_orderdate,1994-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-10-01)) AND (o_orderdate#X < 1994-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (5) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] @@ -123,9 +123,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_returnflag), EqualTo(l_returnflag,R), IsNotNull(l_orderkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] +Arguments: ((isnotnull(l_returnflag#X) AND (l_returnflag#X = R)) AND isnotnull(l_orderkey#X)) (15) ProjectExecTransformer Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -166,9 +166,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(24) NoopFilter +(24) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (25) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt index d4e2cf19c858..2bf02786562e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt @@ -19,18 +19,18 @@ AdaptiveSparkPlan (60) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (20) :- ^ ProjectExecTransformer (11) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) - : :- ^ NoopFilter (2) + : :- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (9) : +- BroadcastQueryStage (7), Statistics(X) : +- ColumnarBroadcastExchange (6) - : +- ^ NoopFilter (4) + : +- ^ FilterExecTransformer (4) : +- ^ Scan parquet (3) +- ^ InputIteratorTransformer (19) +- BroadcastQueryStage (17), Statistics(X) +- ColumnarBroadcastExchange (16) +- ^ ProjectExecTransformer (14) - +- ^ NoopFilter (13) + +- ^ FilterExecTransformer (13) +- ^ Scan parquet (12) +- == Initial Plan == Sort (59) @@ -61,9 +61,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (3) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] @@ -72,9 +72,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (5) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -111,9 +111,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(13) NoopFilter +(13) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) (14) ProjectExecTransformer Output [1]: [n_nationkey#X] @@ -340,7 +340,7 @@ AdaptiveSparkPlan (102) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) :- ^ ProjectExecTransformer (68) : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (67) - : :- ^ NoopFilter (62) + : :- ^ FilterExecTransformer (62) : : +- ^ Scan parquet (61) : +- ^ InputIteratorTransformer (66) : +- BroadcastQueryStage (64), Statistics(X) @@ -374,9 +374,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (63) ReusedExchange [Reuses operator id: 6] Output [2]: [s_suppkey#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt index c2f8ff788955..b796cfae2665 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt @@ -19,10 +19,10 @@ AdaptiveSparkPlan (44) :- ^ InputIteratorTransformer (7) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ ProjectExecTransformer (10) - +- ^ NoopFilter (9) + +- ^ FilterExecTransformer (9) +- ^ Scan parquet (8) +- == Initial Plan == Sort (43) @@ -47,9 +47,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderpriority#X] +Arguments: isnotnull(o_orderkey#X) (3) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_orderpriority#X] @@ -76,9 +76,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_shipdate), In(l_shipmode, [MAIL,SHIP]), GreaterThanOrEqual(l_receiptdate,1994-01-01), LessThan(l_receiptdate,1995-01-01), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] -Arguments: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] +Arguments: ((((((((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND isnotnull(l_shipdate#X)) AND l_shipmode#X IN (MAIL,SHIP)) AND (l_commitdate#X < l_receiptdate#X)) AND (l_shipdate#X < l_commitdate#X)) AND (l_receiptdate#X >= 1994-01-01)) AND (l_receiptdate#X < 1995-01-01)) AND isnotnull(l_orderkey#X)) (10) ProjectExecTransformer Output [2]: [l_orderkey#X, l_shipmode#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt index 0636592d586e..3b7ac85bf23d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (55) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ ProjectExecTransformer (4) - +- ^ NoopFilter (3) + +- ^ FilterExecTransformer (3) +- ^ Scan parquet (2) +- == Initial Plan == Sort (54) @@ -62,9 +62,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(3) NoopFilter +(3) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -Arguments: [o_orderkey#X, o_custkey#X, o_comment#X] +Arguments: ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) (4) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt index cd820d5b3063..7f4949cf7288 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt @@ -12,12 +12,12 @@ AdaptiveSparkPlan (35) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == HashAggregate (34) @@ -40,9 +40,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) (3) ProjectExecTransformer Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] @@ -55,9 +55,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: isnotnull(p_partkey#X) (6) WholeStageCodegenTransformer (X) Input [2]: [p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt index f3e19f99de05..5fa837e185f0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt @@ -11,7 +11,7 @@ AdaptiveSparkPlan (43) :- ^ InputIteratorTransformer (7) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) @@ -22,7 +22,7 @@ AdaptiveSparkPlan (43) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) - +- ^ NoopFilter (9) + +- ^ FilterExecTransformer (9) +- ^ Scan parquet (8) +- == Initial Plan == Sort (42) @@ -48,9 +48,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] +Arguments: isnotnull(s_suppkey#X) (3) WholeStageCodegenTransformer (X) Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -77,9 +77,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01), IsNotNull(l_suppkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) AND isnotnull(l_suppkey#X)) (10) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, (l_extendedprice#X * (1 - l_discount#X)) AS _pre_X#X] @@ -253,7 +253,7 @@ AdaptiveSparkPlan (69) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) - +- ^ NoopFilter (45) + +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) +- == Initial Plan == HashAggregate (68) @@ -273,9 +273,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) (46) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, (l_extendedprice#X * (1 - l_discount#X)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt index f05bc50df0b2..a6302f1beeb5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt @@ -23,12 +23,12 @@ AdaptiveSparkPlan (59) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) - :- ^ NoopFilter (2) + :- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) - +- ^ NoopFilter (4) + +- ^ FilterExecTransformer (4) +- ^ Scan parquet (3) +- == Initial Plan == Sort (58) @@ -60,9 +60,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -Arguments: [ps_partkey#X, ps_suppkey#X] +Arguments: isnotnull(ps_partkey#X) (3) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -71,9 +71,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_type), Not(EqualTo(p_brand,Brand#X)), Not(StringStartsWith(p_type,MEDIUM POLISHED)), In(p_size, [14,19,23,3,36,45,49,9]), IsNotNull(p_partkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: [p_partkey#X, p_brand#X, p_type#X, p_size#X] +Arguments: (((((isnotnull(p_brand#X) AND isnotnull(p_type#X)) AND NOT (p_brand#X = Brand#X)) AND NOT StartsWith(p_type#X, MEDIUM POLISHED)) AND p_size#X IN (49,14,23,45,19,3,36,9)) AND isnotnull(p_partkey#X)) (5) WholeStageCodegenTransformer (X) Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt index 1ee0d23b567f..029c44b1547f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt @@ -17,10 +17,10 @@ AdaptiveSparkPlan (88) : :- ^ InputIteratorTransformer (7) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) - : :- ^ NoopFilter (9) + : :- ^ FilterExecTransformer (9) : : +- ^ Scan parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (88) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) - :- ^ NoopFilter (31) + :- ^ FilterExecTransformer (31) : +- ^ Scan parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) @@ -86,9 +86,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_name#X] -Arguments: [c_custkey#X, c_name#X] +Arguments: isnotnull(c_custkey#X) (3) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_name#X] @@ -115,9 +115,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] +Arguments: (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) (10) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] @@ -214,9 +214,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(31) NoopFilter +(31) FilterExecTransformer Input [2]: [l_orderkey#X, l_quantity#X] -Arguments: [l_orderkey#X, l_quantity#X] +Arguments: isnotnull(l_orderkey#X) (32) ReusedExchange [Reuses operator id: 23] Output [1]: [l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt index 1babbc07b296..069500cf6bdf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt @@ -11,12 +11,12 @@ AdaptiveSparkPlan (34) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == HashAggregate (33) @@ -39,9 +39,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] +Arguments: ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) (3) ProjectExecTransformer Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -54,9 +54,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_size#X, p_container#X] +Arguments: (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) (6) WholeStageCodegenTransformer (X) Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt index 5d6c218366b6..3e2b19835441 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (98) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (55) :- ^ ProjectExecTransformer (46) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (45) - : :- ^ NoopFilter (2) + : :- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (44) : +- BroadcastQueryStage (42), Statistics(X) @@ -21,13 +21,13 @@ AdaptiveSparkPlan (98) : : +- BroadcastQueryStage (16), Statistics(X) : : +- ColumnarBroadcastExchange (15) : : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (13) - : : :- ^ NoopFilter (4) + : : :- ^ FilterExecTransformer (4) : : : +- ^ Scan parquet (3) : : +- ^ InputIteratorTransformer (12) : : +- BroadcastQueryStage (10), Statistics(X) : : +- ColumnarBroadcastExchange (9) : : +- ^ ProjectExecTransformer (7) - : : +- ^ NoopFilter (6) + : : +- ^ FilterExecTransformer (6) : : +- ^ Scan parquet (5) : +- ^ FilterExecTransformer (37) : +- ^ ProjectExecTransformer (36) @@ -40,7 +40,7 @@ AdaptiveSparkPlan (98) : +- ^ FlushableHashAggregateExecTransformer (27) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (26) : :- ^ ProjectExecTransformer (21) - : : +- ^ NoopFilter (20) + : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (25) : +- BroadcastQueryStage (23), Statistics(X) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (98) +- BroadcastQueryStage (52), Statistics(X) +- ColumnarBroadcastExchange (51) +- ^ ProjectExecTransformer (49) - +- ^ NoopFilter (48) + +- ^ FilterExecTransformer (48) +- ^ Scan parquet (47) +- == Initial Plan == Sort (97) @@ -96,9 +96,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] +Arguments: isnotnull(s_nationkey#X) (3) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -107,9 +107,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(4) NoopFilter +(4) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] +Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) (5) Scan parquet Output [2]: [p_partkey#X, p_name#X] @@ -118,9 +118,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(6) NoopFilter +(6) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) (7) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -175,9 +175,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(20) NoopFilter +(20) FilterExecTransformer Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -Arguments: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) (21) ProjectExecTransformer Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] @@ -295,9 +295,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(48) NoopFilter +(48) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) (49) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt index 0b4fa0af4b7d..c93cb695dbd9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt @@ -18,12 +18,12 @@ AdaptiveSparkPlan (92) : : :- ^ InputIteratorTransformer (7) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) @@ -33,19 +33,19 @@ AdaptiveSparkPlan (92) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) - : : +- ^ NoopFilter (19) + : : +- ^ FilterExecTransformer (19) : : +- ^ Scan parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) - : +- ^ NoopFilter (30) + : +- ^ FilterExecTransformer (30) : +- ^ Scan parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) - +- ^ NoopFilter (40) + +- ^ FilterExecTransformer (40) +- ^ Scan parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) @@ -89,9 +89,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] @@ -118,9 +118,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (10) ProjectExecTransformer Output [2]: [l_orderkey#X, l_suppkey#X] @@ -163,9 +163,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] ReadSchema: struct -(19) NoopFilter +(19) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) (20) ProjectExecTransformer Output [2]: [l_orderkey#X, l_suppkey#X] @@ -212,9 +212,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(30) NoopFilter +(30) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderstatus#X] -Arguments: [o_orderkey#X, o_orderstatus#X] +Arguments: ((isnotnull(o_orderstatus#X) AND (o_orderstatus#X = F)) AND isnotnull(o_orderkey#X)) (31) ProjectExecTransformer Output [1]: [o_orderkey#X] @@ -255,9 +255,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(40) NoopFilter +(40) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = SAUDI ARABIA)) AND isnotnull(n_nationkey#X)) (41) ProjectExecTransformer Output [1]: [n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt index 161ac0ada7ec..b5450682267e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (40) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) - :- ^ NoopFilter (2) + :- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) @@ -43,9 +43,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: [c_custkey#X, c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) (3) Scan parquet Output [1]: [o_custkey#X] @@ -217,7 +217,7 @@ Arguments: isFinalPlan=true ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#X, [id=#X] AdaptiveSparkPlan (60) +- == Final Plan == VeloxColumnarToRowExec (53) @@ -228,7 +228,7 @@ AdaptiveSparkPlan (60) +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) - +- ^ NoopFilter (42) + +- ^ FilterExecTransformer (42) +- ^ Scan parquet (41) +- == Initial Plan == HashAggregate (59) @@ -246,9 +246,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal), GreaterThan(c_acctbal,0.00)] ReadSchema: struct -(42) NoopFilter +(42) FilterExecTransformer Input [2]: [c_phone#X, c_acctbal#X] -Arguments: [c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND (c_acctbal#X > 0.00)) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) (43) ProjectExecTransformer Output [1]: [c_acctbal#X] @@ -332,4 +332,25 @@ Results [1]: [avg(c_acctbal#X)#X AS avg(c_acctbal)#X] (60) AdaptiveSparkPlan Output [1]: [avg(c_acctbal)#X] -Arguments: isFinalPlan=true \ No newline at end of file +Arguments: isFinalPlan=true + +Subquery:2 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +AdaptiveSparkPlan (60) ++- == Final Plan == + VeloxColumnarToRowExec (53) + +- ^ RegularHashAggregateExecTransformer (51) + +- ^ InputIteratorTransformer (50) + +- ShuffleQueryStage (48), Statistics(X) + +- ColumnarExchange (47) + +- VeloxResizeBatches (46) + +- ^ FlushableHashAggregateExecTransformer (44) + +- ^ ProjectExecTransformer (43) + +- ^ FilterExecTransformer (42) + +- ^ Scan parquet (41) ++- == Initial Plan == + HashAggregate (59) + +- Exchange (58) + +- HashAggregate (57) + +- Project (56) + +- Filter (55) + +- Scan parquet (54) \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt index c247160f6792..a1635e742a6b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt @@ -19,15 +19,15 @@ AdaptiveSparkPlan (54) : : +- BroadcastQueryStage (6), Statistics(X) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) - : +- ^ NoopFilter (10) + : +- ^ FilterExecTransformer (10) : +- ^ Scan parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) - +- ^ NoopFilter (14) + +- ^ FilterExecTransformer (14) +- ^ Scan parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) @@ -57,9 +57,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_mktsegment#X] -Arguments: [c_custkey#X, c_mktsegment#X] +Arguments: ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) (3) ProjectExecTransformer Output [1]: [c_custkey#X] @@ -90,9 +90,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] +Arguments: (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [c_custkey#X] @@ -111,9 +111,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) (15) ProjectExecTransformer Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt index f4c04d19e78d..4a26f959b7bb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt @@ -17,13 +17,13 @@ AdaptiveSparkPlan (46) +- ^ ProjectExecTransformer (13) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) - +- ^ NoopFilter (5) + +- ^ FilterExecTransformer (5) +- ^ Scan parquet (4) +- == Initial Plan == Sort (45) @@ -49,9 +49,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] +Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_orderdate#X < 1993-10-01)) (3) ProjectExecTransformer Output [2]: [o_orderkey#X, o_orderpriority#X] @@ -64,9 +64,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] ReadSchema: struct -(5) NoopFilter +(5) FilterExecTransformer Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND (l_commitdate#X < l_receiptdate#X)) (6) ProjectExecTransformer Output [1]: [l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt index 5dc04bd48a57..3ba742ae5b6a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt @@ -27,31 +27,31 @@ AdaptiveSparkPlan (102) : : : : :- ^ InputIteratorTransformer (7) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ ProjectExecTransformer (10) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) - : : : +- ^ NoopFilter (14) + : : : +- ^ FilterExecTransformer (14) : : : +- ^ Scan parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) - : : +- ^ NoopFilter (23) + : : +- ^ FilterExecTransformer (23) : : +- ^ Scan parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) - : +- ^ NoopFilter (32) + : +- ^ FilterExecTransformer (32) : +- ^ Scan parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) - +- ^ NoopFilter (41) + +- ^ FilterExecTransformer (41) +- ^ Scan parquet (40) +- == Initial Plan == Sort (101) @@ -97,9 +97,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -126,9 +126,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1994-01-01)) AND (o_orderdate#X < 1995-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (10) ProjectExecTransformer Output [2]: [o_orderkey#X, o_custkey#X] @@ -151,9 +151,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: (isnotnull(l_orderkey#X) AND isnotnull(l_suppkey#X)) (15) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -190,9 +190,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (24) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -229,9 +229,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_name#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (33) WholeStageCodegenTransformer (X) Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] @@ -268,9 +268,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,ASIA), IsNotNull(r_regionkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = ASIA)) AND isnotnull(r_regionkey#X)) (42) ProjectExecTransformer Output [1]: [r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt index f8169a34a98f..fa37c656a1aa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt @@ -9,7 +9,7 @@ AdaptiveSparkPlan (20) +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == HashAggregate (19) @@ -27,9 +27,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), IsNotNull(l_discount), IsNotNull(l_quantity), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), GreaterThanOrEqual(l_discount,0.05), LessThanOrEqual(l_discount,0.07), LessThan(l_quantity,24.00)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((((((isnotnull(l_shipdate#X) AND isnotnull(l_discount#X)) AND isnotnull(l_quantity#X)) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND (l_discount#X >= 0.05)) AND (l_discount#X <= 0.07)) AND (l_quantity#X < 24.00)) (3) ProjectExecTransformer Output [3]: [l_extendedprice#X, l_discount#X, (l_extendedprice#X * l_discount#X) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt index d80dbd225fa8..ab8abd6df364 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt @@ -27,24 +27,24 @@ AdaptiveSparkPlan (95) : : : : :- ^ InputIteratorTransformer (7) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) - : : : : +- ^ NoopFilter (9) + : : : : +- ^ FilterExecTransformer (9) : : : : +- ^ Scan parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) - : : : +- ^ NoopFilter (13) + : : : +- ^ FilterExecTransformer (13) : : : +- ^ Scan parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) - : : +- ^ NoopFilter (22) + : : +- ^ FilterExecTransformer (22) : : +- ^ Scan parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) - : +- ^ NoopFilter (31) + : +- ^ FilterExecTransformer (31) : +- ^ Scan parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) @@ -91,9 +91,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -120,9 +120,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-01-01), LessThanOrEqual(l_shipdate,1996-12-31), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(9) NoopFilter +(9) FilterExecTransformer Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-01-01)) AND (l_shipdate#X <= 1996-12-31)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (10) BroadcastHashJoinExecTransformer Left keys [1]: [s_suppkey#X] @@ -141,9 +141,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(13) NoopFilter +(13) FilterExecTransformer Input [2]: [o_orderkey#X, o_custkey#X] -Arguments: [o_orderkey#X, o_custkey#X] +Arguments: (isnotnull(o_orderkey#X) AND isnotnull(o_custkey#X)) (14) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_custkey#X] @@ -180,9 +180,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(22) NoopFilter +(22) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (23) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -219,9 +219,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), Or(EqualTo(n_name,FRANCE),EqualTo(n_name,GERMANY))] ReadSchema: struct -(31) NoopFilter +(31) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: (isnotnull(n_nationkey#X) AND ((n_name#X = FRANCE) OR (n_name#X = GERMANY))) (32) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt index 4f035d06361d..e075a7b33885 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt @@ -33,40 +33,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- BroadcastQueryStage (6), Statistics(X) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) - : : : : : : : +- ^ NoopFilter (2) + : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) - : : : : : : +- ^ NoopFilter (10) + : : : : : : +- ^ FilterExecTransformer (10) : : : : : : +- ^ Scan parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) - : : : : : +- ^ NoopFilter (14) + : : : : : +- ^ FilterExecTransformer (14) : : : : : +- ^ Scan parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) - : : : : +- ^ NoopFilter (23) + : : : : +- ^ FilterExecTransformer (23) : : : : +- ^ Scan parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) - : : : +- ^ NoopFilter (32) + : : : +- ^ FilterExecTransformer (32) : : : +- ^ Scan parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) - : : +- ^ NoopFilter (41) + : : +- ^ FilterExecTransformer (41) : : +- ^ Scan parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) - : +- ^ NoopFilter (50) + : +- ^ FilterExecTransformer (50) : +- ^ Scan parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) - +- ^ NoopFilter (59) + +- ^ FilterExecTransformer (59) +- ^ Scan parquet (58) +- == Initial Plan == Sort (130) @@ -122,9 +122,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_type), EqualTo(p_type,ECONOMY ANODIZED STEEL), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: ((isnotnull(p_type#X) AND (p_type#X = ECONOMY ANODIZED STEEL)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -155,9 +155,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [p_partkey#X] @@ -176,9 +176,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (15) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -215,9 +215,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1995-01-01), LessThanOrEqual(o_orderdate,1996-12-31), IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1995-01-01)) AND (o_orderdate#X <= 1996-12-31)) AND isnotnull(o_orderkey#X)) AND isnotnull(o_custkey#X)) (24) WholeStageCodegenTransformer (X) Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -254,9 +254,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (33) WholeStageCodegenTransformer (X) Input [2]: [c_custkey#X, c_nationkey#X] @@ -293,9 +293,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [n_nationkey#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (42) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_regionkey#X] @@ -332,9 +332,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(50) NoopFilter +(50) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (51) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] @@ -371,9 +371,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AMERICA), IsNotNull(r_regionkey)] ReadSchema: struct -(59) NoopFilter +(59) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = AMERICA)) AND isnotnull(r_regionkey#X)) (60) ProjectExecTransformer Output [1]: [r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt index deaf8f857e67..8ba39a425d25 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt @@ -28,29 +28,29 @@ AdaptiveSparkPlan (100) : : : : : +- BroadcastQueryStage (6), Statistics(X) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) - : : : : +- ^ NoopFilter (10) + : : : : +- ^ FilterExecTransformer (10) : : : : +- ^ Scan parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) - : : : +- ^ NoopFilter (14) + : : : +- ^ FilterExecTransformer (14) : : : +- ^ Scan parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) - : : +- ^ NoopFilter (23) + : : +- ^ FilterExecTransformer (23) : : +- ^ Scan parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) - : +- ^ NoopFilter (32) + : +- ^ FilterExecTransformer (32) : +- ^ Scan parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) - +- ^ NoopFilter (41) + +- ^ FilterExecTransformer (41) +- ^ Scan parquet (40) +- == Initial Plan == Sort (99) @@ -95,9 +95,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringContains(p_name,green), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: ((isnotnull(p_name#X) AND Contains(p_name#X, green)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [1]: [p_partkey#X] @@ -128,9 +128,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (11) BroadcastHashJoinExecTransformer Left keys [1]: [p_partkey#X] @@ -149,9 +149,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(14) NoopFilter +(14) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (15) WholeStageCodegenTransformer (X) Input [2]: [s_suppkey#X, s_nationkey#X] @@ -188,9 +188,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] ReadSchema: struct -(23) NoopFilter +(23) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] +Arguments: (isnotnull(ps_suppkey#X) AND isnotnull(ps_partkey#X)) (24) WholeStageCodegenTransformer (X) Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -227,9 +227,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(32) NoopFilter +(32) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_orderdate#X] +Arguments: isnotnull(o_orderkey#X) (33) WholeStageCodegenTransformer (X) Input [2]: [o_orderkey#X, o_orderdate#X] @@ -266,9 +266,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(41) NoopFilter +(41) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (42) WholeStageCodegenTransformer (X) Input [2]: [n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt index 37622c9e13a7..b1b2a7507c21 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (30) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == Sort (29) @@ -35,9 +35,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-02)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] +Arguments: (isnotnull(l_shipdate#X) AND (l_shipdate#X <= 1998-09-02)) (3) ProjectExecTransformer Output [8]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) AS _pre_X#X, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true)) * promote_precision(cast(CheckOverflow((1.00 + promote_precision(cast(l_tax#X as decimal(13,2)))), DecimalType(13,2), true) as decimal(26,4)))), DecimalType(38,6), true) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt index 0f902d3da7c7..6bb1784778cd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt @@ -30,28 +30,28 @@ AdaptiveSparkPlan (100) : : : +- ColumnarExchange (6) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) - : : +- ^ NoopFilter (11) + : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) - : +- ^ NoopFilter (28) + : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) - +- ^ NoopFilter (45) + +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) @@ -95,9 +95,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) ProjectExecTransformer Output [8]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -132,9 +132,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-10-01), LessThan(o_orderdate,1994-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-10-01)) AND (o_orderdate#X < 1994-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -200,9 +200,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_returnflag), EqualTo(l_returnflag,R), IsNotNull(l_orderkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] +Arguments: ((isnotnull(l_returnflag#X) AND (l_returnflag#X = R)) AND isnotnull(l_orderkey#X)) (29) ProjectExecTransformer Output [4]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -268,9 +268,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (46) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt index 72d04de06970..42e37d72f12d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt @@ -28,21 +28,21 @@ AdaptiveSparkPlan (82) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) - +- ^ NoopFilter (28) + +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) +- == Initial Plan == Sort (81) @@ -79,9 +79,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (3) ProjectExecTransformer Output [5]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -116,9 +116,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (12) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) (29) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt index b9d0cd3457fd..40f651dd347e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt @@ -21,14 +21,14 @@ AdaptiveSparkPlan (55) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (54) @@ -56,9 +56,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderpriority#X] +Arguments: isnotnull(o_orderkey#X) (3) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -93,9 +93,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_shipdate), In(l_shipmode, [MAIL,SHIP]), GreaterThanOrEqual(l_receiptdate,1994-01-01), LessThan(l_receiptdate,1995-01-01), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] -Arguments: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] +Arguments: ((((((((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND isnotnull(l_shipdate#X)) AND l_shipmode#X IN (MAIL,SHIP)) AND (l_commitdate#X < l_receiptdate#X)) AND (l_shipdate#X < l_commitdate#X)) AND (l_receiptdate#X >= 1994-01-01)) AND (l_receiptdate#X < 1995-01-01)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_shipmode#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt index af0732c1391a..6c48d7e59901 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (58) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ NoopFilter (10) + +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) +- == Initial Plan == Sort (57) @@ -91,9 +91,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -Arguments: [o_orderkey#X, o_custkey#X, o_comment#X] +Arguments: ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) (11) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt index a5270d5dc57a..b04d8d643fda 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (39) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == HashAggregate (38) @@ -44,9 +44,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) (3) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -81,9 +81,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: isnotnull(p_partkey#X) (12) ProjectExecTransformer Output [3]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt index e1ec46810d6b..83bd88329661 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (50) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (50) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (49) @@ -53,9 +53,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] +Arguments: isnotnull(s_suppkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -90,9 +90,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01), IsNotNull(l_suppkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) AND isnotnull(l_suppkey#X)) (12) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt index c4d8034425c8..4c43ec8f19b6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt @@ -28,14 +28,14 @@ AdaptiveSparkPlan (71) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (70) @@ -70,9 +70,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -Arguments: [ps_partkey#X, ps_suppkey#X] +Arguments: isnotnull(ps_partkey#X) (3) ProjectExecTransformer Output [3]: [hash(ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -107,9 +107,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_type), Not(EqualTo(p_brand,Brand#X)), Not(StringStartsWith(p_type,MEDIUM POLISHED)), In(p_size, [14,19,23,3,36,45,49,9]), IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: [p_partkey#X, p_brand#X, p_type#X, p_size#X] +Arguments: (((((isnotnull(p_brand#X) AND isnotnull(p_type#X)) AND NOT (p_brand#X = Brand#X)) AND NOT StartsWith(p_type#X, MEDIUM POLISHED)) AND p_size#X IN (49,14,23,45,19,3,36,9)) AND isnotnull(p_partkey#X)) (12) ProjectExecTransformer Output [5]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt index 196539e2104a..6a9d19d0defe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt @@ -14,14 +14,14 @@ AdaptiveSparkPlan (63) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (63) +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ NoopFilter (22) + +- ^ FilterExecTransformer (22) +- ^ Scan parquet (21) +- == Initial Plan == HashAggregate (62) @@ -66,9 +66,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X] +Arguments: (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) (3) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -103,9 +103,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [p_partkey#X, p_brand#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_container#X] +Arguments: ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) (12) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -149,9 +149,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(22) NoopFilter +(22) FilterExecTransformer Input [2]: [l_partkey#X, l_quantity#X] -Arguments: [l_partkey#X, l_quantity#X] +Arguments: isnotnull(l_partkey#X) (23) FlushableHashAggregateExecTransformer Input [2]: [l_partkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt index b267b885d65c..587eb37cd4c6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt @@ -18,7 +18,7 @@ AdaptiveSparkPlan (110) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (110) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) - : : +- ^ NoopFilter (11) + : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (110) : +- ColumnarExchange (52) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) - : +- ^ NoopFilter (48) + : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) @@ -106,9 +106,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_name#X] -Arguments: [c_custkey#X, c_name#X] +Arguments: isnotnull(c_custkey#X) (3) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_name#X] @@ -143,9 +143,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] +Arguments: (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [5]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -296,9 +296,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(48) NoopFilter +(48) FilterExecTransformer Input [2]: [l_orderkey#X, l_quantity#X] -Arguments: [l_orderkey#X, l_quantity#X] +Arguments: isnotnull(l_orderkey#X) (49) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt index 7ee4b49b8dfe..75795b3eaab5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt @@ -11,14 +11,14 @@ AdaptiveSparkPlan (38) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == HashAggregate (37) @@ -43,9 +43,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] +Arguments: ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) (3) ProjectExecTransformer Output [5]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -80,9 +80,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_size#X, p_container#X] +Arguments: (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) (12) ProjectExecTransformer Output [5]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt index 6c3340893e18..ebcbc702cf47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt @@ -20,7 +20,7 @@ AdaptiveSparkPlan (146) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) @@ -39,14 +39,14 @@ AdaptiveSparkPlan (146) : : : +- ColumnarExchange (15) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) - : : : +- ^ NoopFilter (11) + : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) - : : +- ^ NoopFilter (20) + : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (146) : : +- ColumnarExchange (41) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) - : : +- ^ NoopFilter (37) + : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46) @@ -73,7 +73,7 @@ AdaptiveSparkPlan (146) +- ColumnarExchange (82) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) - +- ^ NoopFilter (78) + +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) +- == Initial Plan == Sort (145) @@ -134,9 +134,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] +Arguments: isnotnull(s_nationkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -171,9 +171,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] +Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) (12) ProjectExecTransformer Output [4]: [hash(ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -208,9 +208,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(20) NoopFilter +(20) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) (21) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -276,9 +276,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(37) NoopFilter +(37) FilterExecTransformer Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -Arguments: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) (38) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -441,9 +441,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) NoopFilter +(78) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) (79) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt index 91a56bf6b7c5..af2e469119be 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (138) : : : +- ColumnarExchange (6) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) @@ -43,7 +43,7 @@ AdaptiveSparkPlan (138) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) @@ -56,21 +56,21 @@ AdaptiveSparkPlan (138) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) - : : +- ^ NoopFilter (29) + : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) - : +- ^ NoopFilter (54) + : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) - +- ^ NoopFilter (71) + +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) +- == Initial Plan == TakeOrderedAndProject (137) @@ -127,9 +127,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) ProjectExecTransformer Output [4]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -164,9 +164,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -238,9 +238,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] ReadSchema: struct -(29) NoopFilter +(29) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) (30) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -337,9 +337,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(54) NoopFilter +(54) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderstatus#X] -Arguments: [o_orderkey#X, o_orderstatus#X] +Arguments: ((isnotnull(o_orderstatus#X) AND (o_orderstatus#X = F)) AND isnotnull(o_orderkey#X)) (55) ProjectExecTransformer Output [2]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X] @@ -405,9 +405,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(71) NoopFilter +(71) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = SAUDI ARABIA)) AND isnotnull(n_nationkey#X)) (72) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt index 01b1f033b519..7e43e67ab1e6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt @@ -21,7 +21,7 @@ AdaptiveSparkPlan (52) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) @@ -53,9 +53,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: [c_custkey#X, c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) (3) ProjectExecTransformer Output [4]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt index 44aea5e81943..b5d79b581920 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt @@ -19,21 +19,21 @@ AdaptiveSparkPlan (67) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) - +- ^ NoopFilter (28) + +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) @@ -68,9 +68,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_mktsegment#X] -Arguments: [c_custkey#X, c_mktsegment#X] +Arguments: ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) (3) ProjectExecTransformer Output [2]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X] @@ -105,9 +105,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] +Arguments: (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [5]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -173,9 +173,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) (29) ProjectExecTransformer Output [4]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt index c192184acd31..7c07aa013416 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt @@ -21,14 +21,14 @@ AdaptiveSparkPlan (56) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (55) @@ -57,9 +57,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] +Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_orderdate#X < 1993-10-01)) (3) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -94,9 +94,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND (l_commitdate#X < l_receiptdate#X)) (12) ProjectExecTransformer Output [2]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt index 6c4e6aa31ebc..f481ec2d47ef 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt @@ -45,42 +45,42 @@ AdaptiveSparkPlan (156) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) - +- ^ NoopFilter (79) + +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) +- == Initial Plan == Sort (155) @@ -141,9 +141,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -178,9 +178,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1994-01-01)) AND (o_orderdate#X < 1995-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -246,9 +246,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: (isnotnull(l_orderkey#X) AND isnotnull(l_suppkey#X)) (29) ProjectExecTransformer Output [5]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -314,9 +314,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (46) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, s_nationkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -382,9 +382,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_name#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (63) ProjectExecTransformer Output [4]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -450,9 +450,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,ASIA), IsNotNull(r_regionkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = ASIA)) AND isnotnull(r_regionkey#X)) (80) ProjectExecTransformer Output [2]: [hash(r_regionkey#X, 42) AS hash_partition_key#X, r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt index 451b7abd33b4..2b97fd28f147 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt @@ -9,7 +9,7 @@ AdaptiveSparkPlan (20) +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == HashAggregate (19) @@ -27,9 +27,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), IsNotNull(l_discount), IsNotNull(l_quantity), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), GreaterThanOrEqual(l_discount,0.05), LessThanOrEqual(l_discount,0.07), LessThan(l_quantity,24.00)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((((((isnotnull(l_shipdate#X) AND isnotnull(l_discount#X)) AND isnotnull(l_quantity#X)) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND (l_discount#X >= 0.05)) AND (l_discount#X <= 0.07)) AND (l_quantity#X < 24.00)) (3) ProjectExecTransformer Output [3]: [l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(l_extendedprice#X) * promote_precision(l_discount#X)), DecimalType(25,4), true) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt index ba296c641bb9..82d0d75e311f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt @@ -45,35 +45,35 @@ AdaptiveSparkPlan (149) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79) @@ -135,9 +135,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -172,9 +172,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-01-01), LessThanOrEqual(l_shipdate,1996-12-31), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-01-01)) AND (l_shipdate#X <= 1996-12-31)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [6]: [hash(l_suppkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -240,9 +240,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [o_orderkey#X, o_custkey#X] -Arguments: [o_orderkey#X, o_custkey#X] +Arguments: (isnotnull(o_orderkey#X) AND isnotnull(o_custkey#X)) (29) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -308,9 +308,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (46) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -376,9 +376,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), Or(EqualTo(n_name,FRANCE),EqualTo(n_name,GERMANY))] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: (isnotnull(n_nationkey#X) AND ((n_name#X = FRANCE) OR (n_name#X = GERMANY))) (63) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt index f75207dbdca9..6a7d97a628f8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt @@ -58,56 +58,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- ColumnarExchange (6) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) - : : : : : : : +- ^ NoopFilter (2) + : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) - : : : : : : +- ^ NoopFilter (11) + : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) - : : : : : +- ^ NoopFilter (28) + : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) - : : : : +- ^ NoopFilter (45) + : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) - : : : +- ^ NoopFilter (62) + : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) - : : +- ^ NoopFilter (79) + : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) - : +- ^ NoopFilter (96) + : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) - +- ^ NoopFilter (113) + +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) +- == Initial Plan == Sort (206) @@ -184,9 +184,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_type), EqualTo(p_type,ECONOMY ANODIZED STEEL), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: ((isnotnull(p_type#X) AND (p_type#X = ECONOMY ANODIZED STEEL)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -221,9 +221,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [6]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (29) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1995-01-01), LessThanOrEqual(o_orderdate,1996-12-31), IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1995-01-01)) AND (o_orderdate#X <= 1996-12-31)) AND isnotnull(o_orderkey#X)) AND isnotnull(o_custkey#X)) (46) ProjectExecTransformer Output [4]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -425,9 +425,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (63) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -493,9 +493,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [n_nationkey#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (80) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -561,9 +561,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(96) NoopFilter +(96) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (97) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] @@ -629,9 +629,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AMERICA), IsNotNull(r_regionkey)] ReadSchema: struct -(113) NoopFilter +(113) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = AMERICA)) AND isnotnull(r_regionkey#X)) (114) ProjectExecTransformer Output [2]: [hash(r_regionkey#X, 42) AS hash_partition_key#X, r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt index 79d82de9da66..a0f1d9c68a43 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt @@ -45,42 +45,42 @@ AdaptiveSparkPlan (155) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) - +- ^ NoopFilter (79) + +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) +- == Initial Plan == Sort (154) @@ -140,9 +140,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringContains(p_name,green), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: ((isnotnull(p_name#X) AND Contains(p_name#X, green)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -177,9 +177,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [7]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -245,9 +245,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (29) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -313,9 +313,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] +Arguments: (isnotnull(ps_suppkey#X) AND isnotnull(ps_partkey#X)) (46) ProjectExecTransformer Output [4]: [hash(ps_suppkey#X, ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -381,9 +381,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_orderdate#X] +Arguments: isnotnull(o_orderkey#X) (63) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -449,9 +449,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (80) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt index 7e1dc79b38aa..417ddf9de25d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (30) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == Sort (29) @@ -35,9 +35,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-02)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] +Arguments: (isnotnull(l_shipdate#X) AND (l_shipdate#X <= 1998-09-02)) (3) ProjectExecTransformer Output [8]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) AS _pre_X#X, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4))) * promote_precision(cast(CheckOverflow((1.00 + promote_precision(cast(l_tax#X as decimal(13,2)))), DecimalType(13,2)) as decimal(26,4)))), DecimalType(38,6)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt index 0fd13247bf74..e42af1ec4cdc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt @@ -30,28 +30,28 @@ AdaptiveSparkPlan (100) : : : +- ColumnarExchange (6) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) - : : +- ^ NoopFilter (11) + : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) - : +- ^ NoopFilter (28) + : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) - +- ^ NoopFilter (45) + +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) @@ -95,9 +95,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) ProjectExecTransformer Output [8]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -132,9 +132,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-10-01), LessThan(o_orderdate,1994-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-10-01)) AND (o_orderdate#X < 1994-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -200,9 +200,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_returnflag), EqualTo(l_returnflag,R), IsNotNull(l_orderkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] +Arguments: ((isnotnull(l_returnflag#X) AND (l_returnflag#X = R)) AND isnotnull(l_orderkey#X)) (29) ProjectExecTransformer Output [4]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -268,9 +268,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (46) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt index 8e8d87524454..fe2dcc9190df 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt @@ -28,21 +28,21 @@ AdaptiveSparkPlan (82) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) - +- ^ NoopFilter (28) + +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) +- == Initial Plan == Sort (81) @@ -79,9 +79,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (3) ProjectExecTransformer Output [5]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -116,9 +116,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (12) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) (29) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] @@ -443,7 +443,7 @@ AdaptiveSparkPlan (136) : : +- ColumnarExchange (88) : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) - : : +- ^ NoopFilter (84) + : : +- ^ FilterExecTransformer (84) : : +- ^ Scan parquet (83) : +- ^ InputIteratorTransformer (95) : +- ShuffleQueryStage (93), Statistics(X) @@ -482,9 +482,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(84) NoopFilter +(84) FilterExecTransformer Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (85) ProjectExecTransformer Output [4]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt index 883294a4f8cc..2ee35e463cb3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt @@ -21,14 +21,14 @@ AdaptiveSparkPlan (55) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (54) @@ -56,9 +56,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderpriority#X] +Arguments: isnotnull(o_orderkey#X) (3) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -93,9 +93,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_shipdate), In(l_shipmode, [MAIL,SHIP]), GreaterThanOrEqual(l_receiptdate,1994-01-01), LessThan(l_receiptdate,1995-01-01), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] -Arguments: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] +Arguments: ((((((((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND isnotnull(l_shipdate#X)) AND l_shipmode#X IN (MAIL,SHIP)) AND (l_commitdate#X < l_receiptdate#X)) AND (l_shipdate#X < l_commitdate#X)) AND (l_receiptdate#X >= 1994-01-01)) AND (l_receiptdate#X < 1995-01-01)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_shipmode#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt index c035c1d4c060..d59a69766201 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (58) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ NoopFilter (10) + +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) +- == Initial Plan == Sort (57) @@ -91,9 +91,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -Arguments: [o_orderkey#X, o_custkey#X, o_comment#X] +Arguments: ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) (11) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt index bf6acad56fea..e570d1a4cfb0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (39) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == HashAggregate (38) @@ -44,9 +44,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) (3) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -81,9 +81,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: isnotnull(p_partkey#X) (12) ProjectExecTransformer Output [3]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt index 9f0f6936c0e8..51baae5186fa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt @@ -13,7 +13,7 @@ AdaptiveSparkPlan (47) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) @@ -24,7 +24,7 @@ AdaptiveSparkPlan (47) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (46) @@ -52,9 +52,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] +Arguments: isnotnull(s_suppkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -89,9 +89,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01), IsNotNull(l_suppkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) AND isnotnull(l_suppkey#X)) (12) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) AS _pre_X#X] @@ -271,7 +271,7 @@ AdaptiveSparkPlan (73) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) - +- ^ NoopFilter (49) + +- ^ FilterExecTransformer (49) +- ^ Scan parquet (48) +- == Initial Plan == HashAggregate (72) @@ -291,9 +291,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(49) NoopFilter +(49) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) (50) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt index 0490704a3ac0..ab417c50bc7b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt @@ -28,14 +28,14 @@ AdaptiveSparkPlan (71) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (70) @@ -70,9 +70,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -Arguments: [ps_partkey#X, ps_suppkey#X] +Arguments: isnotnull(ps_partkey#X) (3) ProjectExecTransformer Output [3]: [hash(ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -107,9 +107,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_type), Not(EqualTo(p_brand,Brand#X)), Not(StringStartsWith(p_type,MEDIUM POLISHED)), In(p_size, [14,19,23,3,36,45,49,9]), IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: [p_partkey#X, p_brand#X, p_type#X, p_size#X] +Arguments: (((((isnotnull(p_brand#X) AND isnotnull(p_type#X)) AND NOT (p_brand#X = Brand#X)) AND NOT StartsWith(p_type#X, MEDIUM POLISHED)) AND p_size#X IN (49,14,23,45,19,3,36,9)) AND isnotnull(p_partkey#X)) (12) ProjectExecTransformer Output [5]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt index 8aeec8a4ecb4..ad32d4cb586d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt @@ -14,14 +14,14 @@ AdaptiveSparkPlan (63) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (63) +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ NoopFilter (22) + +- ^ FilterExecTransformer (22) +- ^ Scan parquet (21) +- == Initial Plan == HashAggregate (62) @@ -66,9 +66,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X] +Arguments: (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) (3) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -103,9 +103,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [p_partkey#X, p_brand#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_container#X] +Arguments: ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) (12) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -149,9 +149,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(22) NoopFilter +(22) FilterExecTransformer Input [2]: [l_partkey#X, l_quantity#X] -Arguments: [l_partkey#X, l_quantity#X] +Arguments: isnotnull(l_partkey#X) (23) FlushableHashAggregateExecTransformer Input [2]: [l_partkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt index 95c531a92eba..42f398883883 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt @@ -18,7 +18,7 @@ AdaptiveSparkPlan (110) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (110) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) - : : +- ^ NoopFilter (11) + : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (110) : +- ColumnarExchange (52) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) - : +- ^ NoopFilter (48) + : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) @@ -106,9 +106,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_name#X] -Arguments: [c_custkey#X, c_name#X] +Arguments: isnotnull(c_custkey#X) (3) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_name#X] @@ -143,9 +143,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] +Arguments: (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [5]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -296,9 +296,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(48) NoopFilter +(48) FilterExecTransformer Input [2]: [l_orderkey#X, l_quantity#X] -Arguments: [l_orderkey#X, l_quantity#X] +Arguments: isnotnull(l_orderkey#X) (49) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt index abcc6b67af9b..20fea0a01629 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt @@ -11,14 +11,14 @@ AdaptiveSparkPlan (38) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == HashAggregate (37) @@ -43,9 +43,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] +Arguments: ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) (3) ProjectExecTransformer Output [5]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -80,9 +80,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_size#X, p_container#X] +Arguments: (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) (12) ProjectExecTransformer Output [5]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt index 365ccaf20760..fdaed6a1a7e8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (143) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) @@ -38,14 +38,14 @@ AdaptiveSparkPlan (143) : : : +- ColumnarExchange (15) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) - : : : +- ^ NoopFilter (11) + : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) - : : +- ^ NoopFilter (20) + : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (143) : : +- ColumnarExchange (41) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) - : : +- ^ NoopFilter (37) + : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46), Statistics(X) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (143) +- ColumnarExchange (82) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) - +- ^ NoopFilter (78) + +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) +- == Initial Plan == Sort (142) @@ -133,9 +133,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] +Arguments: isnotnull(s_nationkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -170,9 +170,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] +Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) (12) ProjectExecTransformer Output [4]: [hash(ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -207,9 +207,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(20) NoopFilter +(20) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) (21) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -275,9 +275,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(37) NoopFilter +(37) FilterExecTransformer Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -Arguments: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) (38) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -440,9 +440,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) NoopFilter +(78) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) (79) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt index aedeb8ce0a31..557dd678da52 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt @@ -28,7 +28,7 @@ AdaptiveSparkPlan (137) : : : +- ColumnarExchange (6) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (137) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) @@ -55,21 +55,21 @@ AdaptiveSparkPlan (137) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) - : : +- ^ NoopFilter (29) + : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) - : +- ^ NoopFilter (54) + : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) - +- ^ NoopFilter (71) + +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) @@ -126,9 +126,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) ProjectExecTransformer Output [4]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -163,9 +163,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -237,9 +237,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] ReadSchema: struct -(29) NoopFilter +(29) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) (30) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -336,9 +336,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(54) NoopFilter +(54) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderstatus#X] -Arguments: [o_orderkey#X, o_orderstatus#X] +Arguments: ((isnotnull(o_orderstatus#X) AND (o_orderstatus#X = F)) AND isnotnull(o_orderkey#X)) (55) ProjectExecTransformer Output [2]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X] @@ -404,9 +404,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(71) NoopFilter +(71) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = SAUDI ARABIA)) AND isnotnull(n_nationkey#X)) (72) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt index 7a51c8367a7e..edd50a4764be 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt @@ -21,7 +21,7 @@ AdaptiveSparkPlan (52) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) @@ -53,9 +53,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: [c_custkey#X, c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) (3) ProjectExecTransformer Output [4]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -271,7 +271,7 @@ Arguments: isFinalPlan=true ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#X, [id=#X] AdaptiveSparkPlan (72) +- == Final Plan == VeloxColumnarToRowExec (65) @@ -282,7 +282,7 @@ AdaptiveSparkPlan (72) +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) - +- ^ NoopFilter (54) + +- ^ FilterExecTransformer (54) +- ^ Scan parquet (53) +- == Initial Plan == HashAggregate (71) @@ -300,9 +300,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal), GreaterThan(c_acctbal,0.00)] ReadSchema: struct -(54) NoopFilter +(54) FilterExecTransformer Input [2]: [c_phone#X, c_acctbal#X] -Arguments: [c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND (c_acctbal#X > 0.00)) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) (55) ProjectExecTransformer Output [1]: [c_acctbal#X] @@ -386,4 +386,25 @@ Results [1]: [avg(c_acctbal#X)#X AS avg(c_acctbal)#X] (72) AdaptiveSparkPlan Output [1]: [avg(c_acctbal)#X] -Arguments: isFinalPlan=true \ No newline at end of file +Arguments: isFinalPlan=true + +Subquery:2 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +AdaptiveSparkPlan (72) ++- == Final Plan == + VeloxColumnarToRowExec (65) + +- ^ RegularHashAggregateExecTransformer (63) + +- ^ InputIteratorTransformer (62) + +- ShuffleQueryStage (60), Statistics(X) + +- ColumnarExchange (59) + +- VeloxResizeBatches (58) + +- ^ FlushableHashAggregateExecTransformer (56) + +- ^ ProjectExecTransformer (55) + +- ^ FilterExecTransformer (54) + +- ^ Scan parquet (53) ++- == Initial Plan == + HashAggregate (71) + +- Exchange (70) + +- HashAggregate (69) + +- Project (68) + +- Filter (67) + +- Scan parquet (66) \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt index b9304b4b971d..ea8a29d6594f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt @@ -19,21 +19,21 @@ AdaptiveSparkPlan (67) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) - +- ^ NoopFilter (28) + +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) @@ -68,9 +68,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_mktsegment#X] -Arguments: [c_custkey#X, c_mktsegment#X] +Arguments: ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) (3) ProjectExecTransformer Output [2]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X] @@ -105,9 +105,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] +Arguments: (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [5]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -173,9 +173,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) (29) ProjectExecTransformer Output [4]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt index 32081ef1cb06..ca3f2707b8e7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt @@ -21,14 +21,14 @@ AdaptiveSparkPlan (56) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (55) @@ -57,9 +57,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] +Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_orderdate#X < 1993-10-01)) (3) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -94,9 +94,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND (l_commitdate#X < l_receiptdate#X)) (12) ProjectExecTransformer Output [2]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt index 0320703d520c..ccedfac29b46 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt @@ -45,42 +45,42 @@ AdaptiveSparkPlan (156) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) - +- ^ NoopFilter (79) + +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) +- == Initial Plan == Sort (155) @@ -141,9 +141,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -178,9 +178,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1994-01-01)) AND (o_orderdate#X < 1995-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -246,9 +246,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: (isnotnull(l_orderkey#X) AND isnotnull(l_suppkey#X)) (29) ProjectExecTransformer Output [5]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -314,9 +314,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (46) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, s_nationkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -382,9 +382,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_name#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (63) ProjectExecTransformer Output [4]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -450,9 +450,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,ASIA), IsNotNull(r_regionkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = ASIA)) AND isnotnull(r_regionkey#X)) (80) ProjectExecTransformer Output [2]: [hash(r_regionkey#X, 42) AS hash_partition_key#X, r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt index ee04d2159020..bb9d04cdcdc7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt @@ -9,7 +9,7 @@ AdaptiveSparkPlan (20) +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == HashAggregate (19) @@ -27,9 +27,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), IsNotNull(l_discount), IsNotNull(l_quantity), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), GreaterThanOrEqual(l_discount,0.05), LessThanOrEqual(l_discount,0.07), LessThan(l_quantity,24.00)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((((((isnotnull(l_shipdate#X) AND isnotnull(l_discount#X)) AND isnotnull(l_quantity#X)) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND (l_discount#X >= 0.05)) AND (l_discount#X <= 0.07)) AND (l_quantity#X < 24.00)) (3) ProjectExecTransformer Output [3]: [l_extendedprice#X, l_discount#X, CheckOverflow((promote_precision(l_extendedprice#X) * promote_precision(l_discount#X)), DecimalType(25,4)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt index 7a74c3c290e9..046147da8ef8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt @@ -45,35 +45,35 @@ AdaptiveSparkPlan (149) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) @@ -135,9 +135,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -172,9 +172,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-01-01), LessThanOrEqual(l_shipdate,1996-12-31), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-01-01)) AND (l_shipdate#X <= 1996-12-31)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [6]: [hash(l_suppkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -240,9 +240,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [o_orderkey#X, o_custkey#X] -Arguments: [o_orderkey#X, o_custkey#X] +Arguments: (isnotnull(o_orderkey#X) AND isnotnull(o_custkey#X)) (29) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -308,9 +308,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (46) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -376,9 +376,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), Or(EqualTo(n_name,FRANCE),EqualTo(n_name,GERMANY))] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: (isnotnull(n_nationkey#X) AND ((n_name#X = FRANCE) OR (n_name#X = GERMANY))) (63) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt index 40e793695a6f..664d74c61985 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt @@ -58,56 +58,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- ColumnarExchange (6) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) - : : : : : : : +- ^ NoopFilter (2) + : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) - : : : : : : +- ^ NoopFilter (11) + : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) - : : : : : +- ^ NoopFilter (28) + : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) - : : : : +- ^ NoopFilter (45) + : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) - : : : +- ^ NoopFilter (62) + : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) - : : +- ^ NoopFilter (79) + : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) - : +- ^ NoopFilter (96) + : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) - +- ^ NoopFilter (113) + +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) +- == Initial Plan == Sort (206) @@ -184,9 +184,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_type), EqualTo(p_type,ECONOMY ANODIZED STEEL), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: ((isnotnull(p_type#X) AND (p_type#X = ECONOMY ANODIZED STEEL)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -221,9 +221,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [6]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (29) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1995-01-01), LessThanOrEqual(o_orderdate,1996-12-31), IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1995-01-01)) AND (o_orderdate#X <= 1996-12-31)) AND isnotnull(o_orderkey#X)) AND isnotnull(o_custkey#X)) (46) ProjectExecTransformer Output [4]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -425,9 +425,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (63) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -493,9 +493,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [n_nationkey#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (80) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -561,9 +561,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(96) NoopFilter +(96) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (97) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] @@ -629,9 +629,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AMERICA), IsNotNull(r_regionkey)] ReadSchema: struct -(113) NoopFilter +(113) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = AMERICA)) AND isnotnull(r_regionkey#X)) (114) ProjectExecTransformer Output [2]: [hash(r_regionkey#X, 42) AS hash_partition_key#X, r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt index ecf1daa6109e..1ae7a93998dc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt @@ -45,42 +45,42 @@ AdaptiveSparkPlan (155) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) - +- ^ NoopFilter (79) + +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) +- == Initial Plan == Sort (154) @@ -140,9 +140,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringContains(p_name,green), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: ((isnotnull(p_name#X) AND Contains(p_name#X, green)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -177,9 +177,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [7]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -245,9 +245,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (29) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -313,9 +313,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] +Arguments: (isnotnull(ps_suppkey#X) AND isnotnull(ps_partkey#X)) (46) ProjectExecTransformer Output [4]: [hash(ps_suppkey#X, ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -381,9 +381,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_orderdate#X] +Arguments: isnotnull(o_orderkey#X) (63) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -449,9 +449,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (80) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt index 03cab493a6c4..450797d3aefd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (30) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == Sort (29) @@ -35,9 +35,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-02)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] +Arguments: (isnotnull(l_shipdate#X) AND (l_shipdate#X <= 1998-09-02)) (3) ProjectExecTransformer Output [8]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, (l_extendedprice#X * (1 - l_discount#X)) AS _pre_X#X, ((l_extendedprice#X * (1 - l_discount#X)) * (1 + l_tax#X)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt index e1e4f7e748e4..6a296c163eb3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt @@ -30,28 +30,28 @@ AdaptiveSparkPlan (100) : : : +- ColumnarExchange (6) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) - : : +- ^ NoopFilter (11) + : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) - : +- ^ NoopFilter (28) + : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) - +- ^ NoopFilter (45) + +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) @@ -95,9 +95,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) ProjectExecTransformer Output [8]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -132,9 +132,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-10-01), LessThan(o_orderdate,1994-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-10-01)) AND (o_orderdate#X < 1994-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -201,9 +201,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_returnflag), EqualTo(l_returnflag,R), IsNotNull(l_orderkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] +Arguments: ((isnotnull(l_returnflag#X) AND (l_returnflag#X = R)) AND isnotnull(l_orderkey#X)) (29) ProjectExecTransformer Output [4]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -270,9 +270,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (46) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt index 6bf42468036b..8fef547f8d1f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt @@ -28,21 +28,21 @@ AdaptiveSparkPlan (82) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) - +- ^ NoopFilter (28) + +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) +- == Initial Plan == Sort (81) @@ -79,9 +79,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (3) ProjectExecTransformer Output [5]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -116,9 +116,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (12) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -185,9 +185,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) (29) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] @@ -447,7 +447,7 @@ AdaptiveSparkPlan (136) : : +- ColumnarExchange (88) : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) - : : +- ^ NoopFilter (84) + : : +- ^ FilterExecTransformer (84) : : +- ^ Scan parquet (83) : +- ^ InputIteratorTransformer (95) : +- ShuffleQueryStage (93), Statistics(X) @@ -486,9 +486,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(84) NoopFilter +(84) FilterExecTransformer Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] +Arguments: isnotnull(ps_suppkey#X) (85) ProjectExecTransformer Output [4]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt index 6ab47cdeee87..a5ff0ee4b58b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt @@ -21,14 +21,14 @@ AdaptiveSparkPlan (55) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (54) @@ -56,9 +56,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderpriority#X] +Arguments: isnotnull(o_orderkey#X) (3) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -93,9 +93,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_shipdate), In(l_shipmode, [MAIL,SHIP]), GreaterThanOrEqual(l_receiptdate,1994-01-01), LessThan(l_receiptdate,1995-01-01), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] -Arguments: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] +Arguments: ((((((((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND isnotnull(l_shipdate#X)) AND l_shipmode#X IN (MAIL,SHIP)) AND (l_commitdate#X < l_receiptdate#X)) AND (l_shipdate#X < l_commitdate#X)) AND (l_receiptdate#X >= 1994-01-01)) AND (l_receiptdate#X < 1995-01-01)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_shipmode#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt index a887ddc89aa2..f0604101ed62 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (58) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ NoopFilter (10) + +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) +- == Initial Plan == Sort (57) @@ -91,9 +91,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(10) NoopFilter +(10) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -Arguments: [o_orderkey#X, o_custkey#X, o_comment#X] +Arguments: ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) (11) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt index 92dec8827f67..768c18ddac04 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (39) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == HashAggregate (38) @@ -44,9 +44,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) (3) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -81,9 +81,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: isnotnull(p_partkey#X) (12) ProjectExecTransformer Output [3]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt index 68a0bbc91079..c91d8a7f5026 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt @@ -13,7 +13,7 @@ AdaptiveSparkPlan (47) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) @@ -24,7 +24,7 @@ AdaptiveSparkPlan (47) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (46) @@ -52,9 +52,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] +Arguments: isnotnull(s_suppkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -89,9 +89,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01), IsNotNull(l_suppkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) AND isnotnull(l_suppkey#X)) (12) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, (l_extendedprice#X * (1 - l_discount#X)) AS _pre_X#X] @@ -273,7 +273,7 @@ AdaptiveSparkPlan (73) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) - +- ^ NoopFilter (49) + +- ^ FilterExecTransformer (49) +- ^ Scan parquet (48) +- == Initial Plan == HashAggregate (72) @@ -293,9 +293,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(49) NoopFilter +(49) FilterExecTransformer Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) (50) ProjectExecTransformer Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, (l_extendedprice#X * (1 - l_discount#X)) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt index 933eb6c625a8..501e4a122ebe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt @@ -28,14 +28,14 @@ AdaptiveSparkPlan (71) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (70) @@ -70,9 +70,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -Arguments: [ps_partkey#X, ps_suppkey#X] +Arguments: isnotnull(ps_partkey#X) (3) ProjectExecTransformer Output [3]: [hash(ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -107,9 +107,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_type), Not(EqualTo(p_brand,Brand#X)), Not(StringStartsWith(p_type,MEDIUM POLISHED)), In(p_size, [14,19,23,3,36,45,49,9]), IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: [p_partkey#X, p_brand#X, p_type#X, p_size#X] +Arguments: (((((isnotnull(p_brand#X) AND isnotnull(p_type#X)) AND NOT (p_brand#X = Brand#X)) AND NOT StartsWith(p_type#X, MEDIUM POLISHED)) AND p_size#X IN (49,14,23,45,19,3,36,9)) AND isnotnull(p_partkey#X)) (12) ProjectExecTransformer Output [5]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt index 74f04de98e04..b8d72004b185 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt @@ -14,14 +14,14 @@ AdaptiveSparkPlan (63) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (63) +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ NoopFilter (22) + +- ^ FilterExecTransformer (22) +- ^ Scan parquet (21) +- == Initial Plan == HashAggregate (62) @@ -66,9 +66,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X] +Arguments: (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) (3) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -103,9 +103,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [p_partkey#X, p_brand#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_container#X] +Arguments: ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) (12) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -150,9 +150,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(22) NoopFilter +(22) FilterExecTransformer Input [2]: [l_partkey#X, l_quantity#X] -Arguments: [l_partkey#X, l_quantity#X] +Arguments: isnotnull(l_partkey#X) (23) FlushableHashAggregateExecTransformer Input [2]: [l_partkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt index 7fc42261f6dd..2cb8a12dd05a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt @@ -18,7 +18,7 @@ AdaptiveSparkPlan (110) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (110) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) - : : +- ^ NoopFilter (11) + : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (110) : +- ColumnarExchange (52) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) - : +- ^ NoopFilter (48) + : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) @@ -106,9 +106,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_name#X] -Arguments: [c_custkey#X, c_name#X] +Arguments: isnotnull(c_custkey#X) (3) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_name#X] @@ -143,9 +143,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] +Arguments: (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [5]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -298,9 +298,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(48) NoopFilter +(48) FilterExecTransformer Input [2]: [l_orderkey#X, l_quantity#X] -Arguments: [l_orderkey#X, l_quantity#X] +Arguments: isnotnull(l_orderkey#X) (49) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt index e022b9cf6311..37cdd2a85b29 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt @@ -11,14 +11,14 @@ AdaptiveSparkPlan (38) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == HashAggregate (37) @@ -43,9 +43,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -Arguments: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] +Arguments: ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) (3) ProjectExecTransformer Output [5]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -80,9 +80,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: [p_partkey#X, p_brand#X, p_size#X, p_container#X] +Arguments: (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) (12) ProjectExecTransformer Output [5]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt index be57604c31f9..cbb0cdea3405 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (143) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) @@ -38,14 +38,14 @@ AdaptiveSparkPlan (143) : : : +- ColumnarExchange (15) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) - : : : +- ^ NoopFilter (11) + : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) - : : +- ^ NoopFilter (20) + : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (143) : : +- ColumnarExchange (41) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) - : : +- ^ NoopFilter (37) + : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46), Statistics(X) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (143) +- ColumnarExchange (82) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) - +- ^ NoopFilter (78) + +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) +- == Initial Plan == Sort (142) @@ -133,9 +133,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] +Arguments: isnotnull(s_nationkey#X) (3) ProjectExecTransformer Output [5]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -170,9 +170,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] +Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) (12) ProjectExecTransformer Output [4]: [hash(ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -207,9 +207,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(20) NoopFilter +(20) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) (21) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -276,9 +276,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(37) NoopFilter +(37) FilterExecTransformer Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -Arguments: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) (38) ProjectExecTransformer Output [4]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -444,9 +444,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) NoopFilter +(78) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) (79) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt index a7b10521886e..5ec8fe9d155b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt @@ -28,7 +28,7 @@ AdaptiveSparkPlan (137) : : : +- ColumnarExchange (6) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) - : : : +- ^ NoopFilter (2) + : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (137) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) @@ -55,21 +55,21 @@ AdaptiveSparkPlan (137) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) - : : +- ^ NoopFilter (29) + : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) - : +- ^ NoopFilter (54) + : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) - +- ^ NoopFilter (71) + +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) @@ -126,9 +126,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_name#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) ProjectExecTransformer Output [4]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -163,9 +163,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -238,9 +238,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] ReadSchema: struct -(29) NoopFilter +(29) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_receiptdate#X > l_commitdate#X)) (30) ProjectExecTransformer Output [3]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -339,9 +339,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(54) NoopFilter +(54) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderstatus#X] -Arguments: [o_orderkey#X, o_orderstatus#X] +Arguments: ((isnotnull(o_orderstatus#X) AND (o_orderstatus#X = F)) AND isnotnull(o_orderkey#X)) (55) ProjectExecTransformer Output [2]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X] @@ -408,9 +408,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(71) NoopFilter +(71) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: ((isnotnull(n_name#X) AND (n_name#X = SAUDI ARABIA)) AND isnotnull(n_nationkey#X)) (72) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt index 150060e8c04b..fee21bac69fe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt @@ -21,7 +21,7 @@ AdaptiveSparkPlan (52) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) @@ -53,9 +53,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: [c_custkey#X, c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) (3) ProjectExecTransformer Output [4]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -273,7 +273,7 @@ Arguments: isFinalPlan=true ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#X, [id=#X] AdaptiveSparkPlan (72) +- == Final Plan == VeloxColumnarToRowExec (65) @@ -284,7 +284,7 @@ AdaptiveSparkPlan (72) +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) - +- ^ NoopFilter (54) + +- ^ FilterExecTransformer (54) +- ^ Scan parquet (53) +- == Initial Plan == HashAggregate (71) @@ -302,9 +302,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_acctbal), GreaterThan(c_acctbal,0.00)] ReadSchema: struct -(54) NoopFilter +(54) FilterExecTransformer Input [2]: [c_phone#X, c_acctbal#X] -Arguments: [c_phone#X, c_acctbal#X] +Arguments: ((isnotnull(c_acctbal#X) AND (c_acctbal#X > 0.00)) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) (55) ProjectExecTransformer Output [1]: [c_acctbal#X] @@ -388,4 +388,25 @@ Results [1]: [avg(c_acctbal#X)#X AS avg(c_acctbal)#X] (72) AdaptiveSparkPlan Output [1]: [avg(c_acctbal)#X] -Arguments: isFinalPlan=true \ No newline at end of file +Arguments: isFinalPlan=true + +Subquery:2 Hosting operator id = 1 Hosting Expression = Subquery subquery#X, [id=#X] +AdaptiveSparkPlan (72) ++- == Final Plan == + VeloxColumnarToRowExec (65) + +- ^ RegularHashAggregateExecTransformer (63) + +- ^ InputIteratorTransformer (62) + +- ShuffleQueryStage (60), Statistics(X) + +- ColumnarExchange (59) + +- VeloxResizeBatches (58) + +- ^ FlushableHashAggregateExecTransformer (56) + +- ^ ProjectExecTransformer (55) + +- ^ FilterExecTransformer (54) + +- ^ Scan parquet (53) ++- == Initial Plan == + HashAggregate (71) + +- Exchange (70) + +- HashAggregate (69) + +- Project (68) + +- Filter (67) + +- Scan parquet (66) \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt index e3697b056ec0..83064ad3a01d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt @@ -19,21 +19,21 @@ AdaptiveSparkPlan (67) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) - : : +- ^ NoopFilter (2) + : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) - : +- ^ NoopFilter (11) + : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) - +- ^ NoopFilter (28) + +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) @@ -68,9 +68,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_mktsegment#X] -Arguments: [c_custkey#X, c_mktsegment#X] +Arguments: ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) (3) ProjectExecTransformer Output [2]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X] @@ -105,9 +105,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] +Arguments: (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [5]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -174,9 +174,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) (29) ProjectExecTransformer Output [4]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt index 65488adcaca5..3f055feb01b6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt @@ -21,14 +21,14 @@ AdaptiveSparkPlan (56) : +- ColumnarExchange (6) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) - : +- ^ NoopFilter (2) + : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) - +- ^ NoopFilter (11) + +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == Sort (55) @@ -57,9 +57,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -Arguments: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] +Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_orderdate#X < 1993-10-01)) (3) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -94,9 +94,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] -Arguments: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] +Arguments: ((isnotnull(l_commitdate#X) AND isnotnull(l_receiptdate#X)) AND (l_commitdate#X < l_receiptdate#X)) (12) ProjectExecTransformer Output [2]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt index 2e872491d4c1..87eaa9b8dce6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt @@ -45,42 +45,42 @@ AdaptiveSparkPlan (156) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) - +- ^ NoopFilter (79) + +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) +- == Initial Plan == Sort (155) @@ -141,9 +141,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (3) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -178,9 +178,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1994-01-01)) AND (o_orderdate#X < 1995-01-01)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) (12) ProjectExecTransformer Output [3]: [hash(o_custkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -247,9 +247,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: (isnotnull(l_orderkey#X) AND isnotnull(l_suppkey#X)) (29) ProjectExecTransformer Output [5]: [hash(l_orderkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -316,9 +316,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (46) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, s_nationkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -385,9 +385,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_name#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (63) ProjectExecTransformer Output [4]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -454,9 +454,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,ASIA), IsNotNull(r_regionkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = ASIA)) AND isnotnull(r_regionkey#X)) (80) ProjectExecTransformer Output [2]: [hash(r_regionkey#X, 42) AS hash_partition_key#X, r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt index f8169a34a98f..fa37c656a1aa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt @@ -9,7 +9,7 @@ AdaptiveSparkPlan (20) +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) - +- ^ NoopFilter (2) + +- ^ FilterExecTransformer (2) +- ^ Scan parquet (1) +- == Initial Plan == HashAggregate (19) @@ -27,9 +27,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), IsNotNull(l_discount), IsNotNull(l_quantity), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), GreaterThanOrEqual(l_discount,0.05), LessThanOrEqual(l_discount,0.07), LessThan(l_quantity,24.00)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: (((((((isnotnull(l_shipdate#X) AND isnotnull(l_discount#X)) AND isnotnull(l_quantity#X)) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND (l_discount#X >= 0.05)) AND (l_discount#X <= 0.07)) AND (l_quantity#X < 24.00)) (3) ProjectExecTransformer Output [3]: [l_extendedprice#X, l_discount#X, (l_extendedprice#X * l_discount#X) AS _pre_X#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt index 9f03472f4bf5..4f781dd6d07e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt @@ -45,35 +45,35 @@ AdaptiveSparkPlan (149) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) @@ -135,9 +135,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (3) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -172,9 +172,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-01-01), LessThanOrEqual(l_shipdate,1996-12-31), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] +Arguments: ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-01-01)) AND (l_shipdate#X <= 1996-12-31)) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [6]: [hash(l_suppkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -241,9 +241,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [o_orderkey#X, o_custkey#X] -Arguments: [o_orderkey#X, o_custkey#X] +Arguments: (isnotnull(o_orderkey#X) AND isnotnull(o_custkey#X)) (29) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -310,9 +310,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (46) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -379,9 +379,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), Or(EqualTo(n_name,FRANCE),EqualTo(n_name,GERMANY))] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: (isnotnull(n_nationkey#X) AND ((n_name#X = FRANCE) OR (n_name#X = GERMANY))) (63) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt index e858f7a1e999..ae71cca0e43c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt @@ -58,56 +58,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- ColumnarExchange (6) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) - : : : : : : : +- ^ NoopFilter (2) + : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) - : : : : : : +- ^ NoopFilter (11) + : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) - : : : : : +- ^ NoopFilter (28) + : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) - : : : : +- ^ NoopFilter (45) + : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) - : : : +- ^ NoopFilter (62) + : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) - : : +- ^ NoopFilter (79) + : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) - : +- ^ NoopFilter (96) + : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) - +- ^ NoopFilter (113) + +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) +- == Initial Plan == Sort (206) @@ -184,9 +184,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_type), EqualTo(p_type,ECONOMY ANODIZED STEEL), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_type#X] -Arguments: [p_partkey#X, p_type#X] +Arguments: ((isnotnull(p_type#X) AND (p_type#X = ECONOMY ANODIZED STEEL)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -221,9 +221,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [6]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -290,9 +290,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (29) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -359,9 +359,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1995-01-01), LessThanOrEqual(o_orderdate,1996-12-31), IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_custkey#X, o_orderdate#X] +Arguments: ((((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1995-01-01)) AND (o_orderdate#X <= 1996-12-31)) AND isnotnull(o_orderkey#X)) AND isnotnull(o_custkey#X)) (46) ProjectExecTransformer Output [4]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -428,9 +428,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [c_custkey#X, c_nationkey#X] -Arguments: [c_custkey#X, c_nationkey#X] +Arguments: (isnotnull(c_custkey#X) AND isnotnull(c_nationkey#X)) (63) ProjectExecTransformer Output [3]: [hash(c_custkey#X, 42) AS hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -497,9 +497,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [n_nationkey#X, n_regionkey#X] -Arguments: [n_nationkey#X, n_regionkey#X] +Arguments: (isnotnull(n_nationkey#X) AND isnotnull(n_regionkey#X)) (80) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -566,9 +566,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(96) NoopFilter +(96) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (97) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] @@ -635,9 +635,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AMERICA), IsNotNull(r_regionkey)] ReadSchema: struct -(113) NoopFilter +(113) FilterExecTransformer Input [2]: [r_regionkey#X, r_name#X] -Arguments: [r_regionkey#X, r_name#X] +Arguments: ((isnotnull(r_name#X) AND (r_name#X = AMERICA)) AND isnotnull(r_regionkey#X)) (114) ProjectExecTransformer Output [2]: [hash(r_regionkey#X, 42) AS hash_partition_key#X, r_regionkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt index 39b2996cde0e..a26c30f45f85 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt @@ -45,42 +45,42 @@ AdaptiveSparkPlan (155) : : : : : +- ColumnarExchange (6) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) - : : : : : +- ^ NoopFilter (2) + : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) - : : : : +- ^ NoopFilter (11) + : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) - : : : +- ^ NoopFilter (28) + : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) - : : +- ^ NoopFilter (45) + : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) - : +- ^ NoopFilter (62) + : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) - +- ^ NoopFilter (79) + +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) +- == Initial Plan == Sort (154) @@ -140,9 +140,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringContains(p_name,green), IsNotNull(p_partkey)] ReadSchema: struct -(2) NoopFilter +(2) FilterExecTransformer Input [2]: [p_partkey#X, p_name#X] -Arguments: [p_partkey#X, p_name#X] +Arguments: ((isnotnull(p_name#X) AND Contains(p_name#X, green)) AND isnotnull(p_partkey#X)) (3) ProjectExecTransformer Output [2]: [hash(p_partkey#X, 42) AS hash_partition_key#X, p_partkey#X] @@ -177,9 +177,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] ReadSchema: struct -(11) NoopFilter +(11) FilterExecTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] +Arguments: ((isnotnull(l_partkey#X) AND isnotnull(l_suppkey#X)) AND isnotnull(l_orderkey#X)) (12) ProjectExecTransformer Output [7]: [hash(l_partkey#X, 42) AS hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -246,9 +246,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(28) NoopFilter +(28) FilterExecTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -Arguments: [s_suppkey#X, s_nationkey#X] +Arguments: (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) (29) ProjectExecTransformer Output [3]: [hash(s_suppkey#X, 42) AS hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -315,9 +315,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] ReadSchema: struct -(45) NoopFilter +(45) FilterExecTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] +Arguments: (isnotnull(ps_suppkey#X) AND isnotnull(ps_partkey#X)) (46) ProjectExecTransformer Output [4]: [hash(ps_suppkey#X, ps_partkey#X, 42) AS hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -384,9 +384,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderkey)] ReadSchema: struct -(62) NoopFilter +(62) FilterExecTransformer Input [2]: [o_orderkey#X, o_orderdate#X] -Arguments: [o_orderkey#X, o_orderdate#X] +Arguments: isnotnull(o_orderkey#X) (63) ProjectExecTransformer Output [3]: [hash(o_orderkey#X, 42) AS hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -453,9 +453,9 @@ Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_nationkey)] ReadSchema: struct -(79) NoopFilter +(79) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] -Arguments: [n_nationkey#X, n_name#X] +Arguments: isnotnull(n_nationkey#X) (80) ProjectExecTransformer Output [3]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X, n_name#X] diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala index 8e4580360aa1..f95fd2389143 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} +import org.apache.spark.sql.execution.datasources.v2.FileScan import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.BuildSideRelation import org.apache.spark.sql.execution.metric.SQLMetric @@ -617,9 +617,9 @@ trait SparkPlanExecApi { }) } sparkExecNode match { - case fileSourceScan: FileSourceScanExec => + case fileSourceScan: FileSourceScanExecTransformerBase => getPushedFilter(fileSourceScan.dataFilters) - case batchScan: BatchScanExec => + case batchScan: BatchScanExecTransformerBase => batchScan.scan match { case fileScan: FileScan => getPushedFilter(fileScan.dataFilters) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index aa5b7aea76af..0801ffb27ae3 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -357,28 +357,4 @@ object FilterHandler extends PredicateHelper { */ def getRemainingFilters(scanFilters: Seq[Expression], filters: Seq[Expression]): Seq[Expression] = (filters.toSet -- scanFilters.toSet).toSeq - - // Separate and compare the filter conditions in Scan and Filter. - // Try to push down the remaining conditions in Filter into Scan. - def pushFilterToScan(condition: Expression, scan: SparkPlan): SparkPlan = - scan match { - case fileSourceScan: FileSourceScanExec => - val pushDownFilters = - BackendsApiManager.getSparkPlanExecApiInstance.postProcessPushDownFilter( - splitConjunctivePredicates(condition), - fileSourceScan) - ScanTransformerFactory.createFileSourceScanTransformer( - fileSourceScan, - allPushDownFilters = Some(pushDownFilters)) - case batchScan: BatchScanExec => - val pushDownFilters = - BackendsApiManager.getSparkPlanExecApiInstance.postProcessPushDownFilter( - splitConjunctivePredicates(condition), - batchScan) - ScanTransformerFactory.createBatchScanTransformer( - batchScan, - allPushDownFilters = Some(pushDownFilters)) - case other => - throw new GlutenNotSupportException(s"${other.getClass.toString} is not supported.") - } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala index 69b74f3a68c8..b74eee3b860e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala @@ -35,7 +35,7 @@ object MiscColumnarRules { object TransformPreOverrides { def apply(): TransformPreOverrides = { TransformPreOverrides( - List(OffloadFilter()), + List(), List( OffloadOthers(), OffloadAggregate(), diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala index 6f1b1fa3b776..2b61bfbc3d9c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala @@ -236,51 +236,6 @@ object OffloadJoin { } } -// Filter transformation. -case class OffloadFilter() extends OffloadSingleNode with LogLevelUtil { - import OffloadOthers._ - private val replace = new ReplaceSingleNode() - - override def offload(plan: SparkPlan): SparkPlan = plan match { - case filter: FilterExec => - genFilterExec(filter) - case other => other - } - - /** - * Generate a plan for filter. - * - * @param filter - * : the original Spark plan. - * @return - * the actually used plan for execution. - */ - private def genFilterExec(filter: FilterExec): SparkPlan = { - if (FallbackTags.nonEmpty(filter)) { - return filter - } - - // FIXME: Filter push-down should be better done by Vanilla Spark's planner or by - // a individual rule. - // Push down the left conditions in Filter into FileSourceScan. - val newChild: SparkPlan = filter.child match { - case scan @ (_: FileSourceScanExec | _: BatchScanExec) => - if (FallbackTags.maybeOffloadable(scan)) { - val newScan = - FilterHandler.pushFilterToScan(filter.condition, scan) - newScan match { - case ts: TransformSupport if ts.doValidate().ok() => ts - case _ => scan - } - } else scan - case _ => filter.child - } - logDebug(s"Columnar Processing for ${filter.getClass} is currently supported.") - BackendsApiManager.getSparkPlanExecApiInstance - .genFilterExecTransformer(filter.condition, newChild) - } -} - // Other transformations. case class OffloadOthers() extends OffloadSingleNode with LogLevelUtil { import OffloadOthers._ @@ -313,6 +268,10 @@ object OffloadOthers { case plan: CoalesceExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") ColumnarCoalesceExec(plan.numPartitions, plan.child) + case plan: FilterExec => + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + BackendsApiManager.getSparkPlanExecApiInstance + .genFilterExecTransformer(plan.condition, plan.child) case plan: ProjectExec => val columnarChild = plan.child logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PushDownFilterToScan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PushDownFilterToScan.scala new file mode 100644 index 000000000000..3ff098ae4b41 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PushDownFilterToScan.scala @@ -0,0 +1,66 @@ +/* + * 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 org.apache.gluten.extension.columnar + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BatchScanExecTransformerBase, FileSourceScanExecTransformer, FilterExecTransformerBase} + +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ + +/** + * Vanilla spark just push down part of filter condition into scan, however gluten can push down all + * filters. + */ +object PushDownFilterToScan extends Rule[SparkPlan] with PredicateHelper { + override def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case filter: FilterExecTransformerBase => + filter.child match { + case fileScan: FileSourceScanExecTransformer => + val pushDownFilters = + BackendsApiManager.getSparkPlanExecApiInstance.postProcessPushDownFilter( + splitConjunctivePredicates(filter.cond), + fileScan) + val newScan = fileScan.copy(dataFilters = pushDownFilters) + if (newScan.doValidate().ok()) { + filter.withNewChildren(Seq(newScan)) + } else { + filter + } + case batchScan: BatchScanExecTransformerBase => + val pushDownFilters = + BackendsApiManager.getSparkPlanExecApiInstance.postProcessPushDownFilter( + splitConjunctivePredicates(filter.cond), + batchScan) + // If BatchScanExecTransformerBase's parent is filter, pushdownFilters can't be None. + batchScan.setPushDownFilters(Seq.empty) + val newScan = batchScan + if (pushDownFilters.size > 0) { + newScan.setPushDownFilters(pushDownFilters) + if (newScan.doValidate().ok()) { + filter.withNewChildren(Seq(newScan)) + } else { + filter + } + } else { + filter + } + case _ => filter + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index 3737dd4af2f4..cf6aeb42c86c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -53,9 +53,7 @@ case class EnumeratedTransform(session: SparkSession, outputsColumnar: Boolean) .build() private val rules = List( - new PushFilterToScan(validator), - RemoveSort, - RemoveFilter + RemoveSort ) // TODO: Should obey ReplaceSingleNode#applyScanNotTransformable to select diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/PushFilterToScan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/PushFilterToScan.scala deleted file mode 100644 index 4070a0a58612..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/PushFilterToScan.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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 org.apache.gluten.extension.columnar.enumerated - -import org.apache.gluten.execution.{FilterHandler, TransformSupport} -import org.apache.gluten.extension.columnar.validator.Validator -import org.apache.gluten.ras.path.Pattern._ -import org.apache.gluten.ras.path.Pattern.Matchers._ -import org.apache.gluten.ras.rule.{RasRule, Shape} -import org.apache.gluten.ras.rule.Shapes._ - -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec - -// TODO: Match on Vanilla filter + Gluten scan. -class PushFilterToScan(validator: Validator) extends RasRule[SparkPlan] { - override def shift(node: SparkPlan): Iterable[SparkPlan] = node match { - case FilterAndScan(filter, scan) => - validator.validate(scan) match { - case Validator.Failed(reason) => - List.empty - case Validator.Passed => - val newScan = - FilterHandler.pushFilterToScan(filter.condition, scan) - newScan match { - case ts: TransformSupport if ts.doValidate().ok() => - List(filter.withNewChildren(List(ts))) - case _ => - List.empty - } - } - case _ => - List.empty - } - - override def shape(): Shape[SparkPlan] = - anyOf( - pattern( - branch[SparkPlan]( - clazz(classOf[FilterExec]), - leaf( - or(clazz(classOf[FileSourceScanExec]), clazz(classOf[BatchScanExec])) - ) - ).build()), - pattern( - branch[SparkPlan]( - clazz(classOf[FilterExec]), - branch( - clazz(classOf[ColumnarToRowTransition]), - leaf( - or(clazz(classOf[FileSourceScanExec]), clazz(classOf[BatchScanExec])) - ) - ) - ).build()) - ) - - private object FilterAndScan { - def unapply(node: SparkPlan): Option[(FilterExec, SparkPlan)] = node match { - case f @ FilterExec(cond, ColumnarToRowExec(scan)) => - ensureScan(scan) - Some(f, scan) - case f @ FilterExec(cond, scan) => - ensureScan(scan) - Some(f, scan) - case _ => - None - } - - private def ensureScan(node: SparkPlan): Unit = { - assert(node.isInstanceOf[FileSourceScanExec] || node.isInstanceOf[BatchScanExec]) - } - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala deleted file mode 100644 index 8b8441e8d6ce..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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 org.apache.gluten.extension.columnar.enumerated - -import org.apache.gluten.execution._ -import org.apache.gluten.metrics.MetricsUpdater -import org.apache.gluten.ras.path.Pattern._ -import org.apache.gluten.ras.path.Pattern.Matchers._ -import org.apache.gluten.ras.rule.{RasRule, Shape} -import org.apache.gluten.ras.rule.Shapes._ -import org.apache.gluten.substrait.SubstraitContext - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} -import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.vectorized.ColumnarBatch - -// Removes Gluten filter operator if its no-op. Typically a Gluten filter is no-op when it -// pushes all of its conditions into the child scan. -// -// The rule is needed in RAS since our cost model treats all filter + scan plans with constant cost -// because the pushed filter is not considered in the model. Removing the filter will make -// optimizer choose a single scan as the winner sub-plan since a single scan's cost is lower than -// filter + scan. -object RemoveFilter extends RasRule[SparkPlan] { - override def shift(node: SparkPlan): Iterable[SparkPlan] = { - val filter = node.asInstanceOf[FilterExecTransformerBase] - if (filter.isNoop()) { - if (filter.output != filter.child.output) { - val out = NoopFilter(filter.child, filter.output) - out.copyTagsFrom(filter) - return List(out) - } - return List(filter.child) - } - List.empty - } - - override def shape(): Shape[SparkPlan] = - pattern( - branch[SparkPlan]( - clazz(classOf[FilterExecTransformerBase]), - leaf(clazz(classOf[BasicScanExecTransformer])) - ).build()) - - // A noop filter placeholder that indicates that all conditions were pushed down to scan. - // - // This operator has zero cost in cost model to avoid planner from choosing the - // original filter-scan that doesn't have all conditions pushed down to scan. - // - // We cannot simply remove the filter to let planner choose the pushed scan since by vanilla - // Spark's definition the filter may have different output nullability than scan. So - // we have to keep this empty filter to let the optimized tree have the identical output schema - // with the original tree. If we simply remove the filter, possible UBs might be caused. For - // example, redundant broadcast exchanges may be added by EnsureRequirements because the - // broadcast join detects that its join keys' nullabilities have been changed. Then AQE - // re-optimization could be broken by ValidateSparkPlan so that AQE could completely - // have no effect as if it's off. This case can be observed by explicitly setting a higher - // AQE logger level to make sure the validation log doesn't get suppressed, e.g., - // spark.sql.adaptive.logLevel=ERROR. - case class NoopFilter(override val child: SparkPlan, override val output: Seq[Attribute]) - extends UnaryTransformSupport { - override def metricsUpdater(): MetricsUpdater = MetricsUpdater.None - override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(newChild) - override def outputPartitioning: Partitioning = child.outputPartitioning - override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override protected def doTransform(context: SubstraitContext): TransformContext = - child.asInstanceOf[TransformSupport].transform(context) - override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala index 86f3f6e0b5e2..9810656197be 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala @@ -16,7 +16,6 @@ */ package org.apache.gluten.planner.cost -import org.apache.gluten.extension.columnar.enumerated.RemoveFilter import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil @@ -33,9 +32,6 @@ class LegacyCostModel extends LongCostModel { // much as possible. override def selfLongCostOf(node: SparkPlan): Long = { node match { - case _: RemoveFilter.NoopFilter => - // To make planner choose the tree that has applied rule PushFilterToScan. - 0L case ColumnarToRowExec(_) => 10L case RowToColumnarExec(_) => 10L case ColumnarToRowLike(_) => 10L diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala index 086bc4c0ce13..9b499b967c3a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala @@ -17,7 +17,6 @@ package org.apache.gluten.planner.cost import org.apache.gluten.execution.RowToColumnarExecBase -import org.apache.gluten.extension.columnar.enumerated.RemoveFilter import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil @@ -30,9 +29,6 @@ class RoughCostModel extends LongCostModel { override def selfLongCostOf(node: SparkPlan): Long = { node match { - case _: RemoveFilter.NoopFilter => - // To make planner choose the tree that has applied rule PushFilterToScan. - 0L case ProjectExec(projectList, _) if projectList.forall(isCheapExpression) => // Make trivial ProjectExec has the same cost as ProjectExecTransform to reduce unnecessary // c2r and r2c.