diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 68db5f8600..56d0e8f579 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -357,7 +357,7 @@ index a9f69ab28a1..760ea0e9565 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -index 433b4741979..07148eee480 100644 +index 433b4741979..2334a7e8264 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -23,8 +23,9 @@ import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} @@ -371,6 +371,15 @@ index 433b4741979..07148eee480 100644 import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ +@@ -981,7 +982,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + } + +- test("SPARK-21258: complex object in combination with spilling") { ++ test("SPARK-21258: complex object in combination with spilling", IgnoreComet("Comet does not support spilling")) { + // Make sure we trigger the spilling path. + withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") { @@ -1186,10 +1187,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest } diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index e7b6e5fffa..8b0df0df49 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -338,7 +338,7 @@ index 7ee18df3756..d09f70e5d99 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -index a1d5d579338..c201d39cc78 100644 +index a1d5d579338..7e0acccd124 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression @@ -352,6 +352,15 @@ index a1d5d579338..c201d39cc78 100644 import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ +@@ -982,7 +983,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + } + +- test("SPARK-21258: complex object in combination with spilling") { ++ test("SPARK-21258: complex object in combination with spilling", IgnoreComet("Comet does not support spilling")) { + // Make sure we trigger the spilling path. + withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") { @@ -1187,10 +1188,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest } diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index 152538270d..f58314b158 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -475,7 +475,7 @@ index 5b88eeefeca..d4f07bc182a 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -index 01e72daead4..0a8d1e8b9b9 100644 +index 01e72daead4..a87d6162610 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression @@ -489,6 +489,15 @@ index 01e72daead4..0a8d1e8b9b9 100644 import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ +@@ -963,7 +964,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + } + +- test("SPARK-21258: complex object in combination with spilling") { ++ test("SPARK-21258: complex object in combination with spilling", IgnoreComet("Comet does not support spilling")) { + // Make sure we trigger the spilling path. + withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") { @@ -1142,10 +1143,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest } @@ -1746,6 +1755,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +index aa619c5cde8..4b5a42e683f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.TestUtils.assertSpilled +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} + import org.apache.spark.sql.test.SharedSparkSession + +@@ -470,7 +470,7 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + +- test("test with low buffer spill threshold") { ++ test("test with low buffer spill threshold", IgnoreComet("Comet does not support spilling")) { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index aed11badb71..1a365b5aacf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala diff --git a/dev/diffs/4.1.1.diff b/dev/diffs/4.1.1.diff index 411034ff70..060139cf35 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.diff @@ -555,7 +555,7 @@ index 5b88eeefeca..d4f07bc182a 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -index 6e9f3385571..15615f3e417 100644 +index 6e9f3385571..72331b0870c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression @@ -569,6 +569,15 @@ index 6e9f3385571..15615f3e417 100644 import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ +@@ -963,7 +964,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + } + +- test("SPARK-21258: complex object in combination with spilling") { ++ test("SPARK-21258: complex object in combination with spilling", IgnoreComet("Comet does not support spilling")) { + // Make sure we trigger the spilling path. + withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") { @@ -1142,10 +1143,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest } @@ -1863,6 +1872,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +index aa619c5cde8..4b5a42e683f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.TestUtils.assertSpilled +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} + import org.apache.spark.sql.test.SharedSparkSession + +@@ -470,7 +470,7 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + +- test("test with low buffer spill threshold") { ++ test("test with low buffer spill threshold", IgnoreComet("Comet does not support spilling")) { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index aed11badb71..1a365b5aacf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 478c7a8d98..331a368cb9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -39,7 +39,7 @@ use datafusion::functions_aggregate::min_max::max_udaf; use datafusion::functions_aggregate::min_max::min_udaf; use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use datafusion::physical_plan::windows::BoundedWindowAggExec; +use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::InputOrderMode; use datafusion::{ arrow::{compute::SortOptions, datatypes::SchemaRef}, @@ -1908,16 +1908,85 @@ impl PhysicalPlanner { }) .collect(); - let window_agg = Arc::new(BoundedWindowAggExec::try_new( - window_expr?, - Arc::clone(&child.native_plan), - InputOrderMode::Sorted, - !partition_exprs.is_empty(), - )?); + // Route to `BoundedWindowAggExec` when every window expression can + // run with bounded memory. This uses DataFusion's + // `evaluate_stateful` / row-by-row `evaluate` path, which is the + // correct implementation for `LEAD` / `LAG` with `IGNORE NULLS` + // (`WindowAggExec` calls `evaluate_all`, whose + // `evaluate_all_with_ignore_null` has a sign-wrap bug for `LEAD` + // that produces all-NULL output). + // + // Fall back to `WindowAggExec` otherwise. That covers + // `PERCENT_RANK` / `CUME_DIST` / `NTILE` + // (`!uses_bounded_memory()` — "Can not execute X in a streaming + // fashion") and keeps the Spark-compatible Comet UDAFs + // (`SumDecimal` / `SumInteger` / `AvgDecimal` / `Avg`) on the + // non-streaming path since they don't implement `retract_batch`. + // Because `process_agg_func` already picks DataFusion's + // retract-capable built-ins for sliding aggregate frames, + // ever-expanding aggregate frames (all that route to + // `BoundedWindowAggExec` as `PlainAggregateWindowExpr`) never + // trigger a retract call. + let window_expr = window_expr?; + let all_bounded = window_expr.iter().all(|e| e.uses_bounded_memory()); + let window_agg: Arc = if all_bounded { + Arc::new(BoundedWindowAggExec::try_new( + window_expr, + Arc::clone(&child.native_plan), + InputOrderMode::Sorted, + !partition_exprs.is_empty(), + )?) + } else { + Arc::new(WindowAggExec::try_new( + window_expr, + Arc::clone(&child.native_plan), + !partition_exprs.is_empty(), + )?) + }; + + // DataFusion's window functions don't always return the same Arrow + // type that Spark expects (e.g. `row_number` returns UInt64 while + // Spark expects Int32). If any window expression carries a + // `result_type` that differs from the actual output type, wrap the + // aggregate in a projection that casts the mismatched columns. + let final_plan: Arc = { + let agg_schema = window_agg.schema(); + let input_field_count = input_schema.fields().len(); + let mut needs_cast = false; + let mut proj_exprs: Vec<(Arc, String)> = + Vec::with_capacity(agg_schema.fields().len()); + for (idx, field) in agg_schema.fields().iter().enumerate() { + let col: Arc = Arc::new(Column::new(field.name(), idx)); + let expr: Arc = if idx >= input_field_count { + let w = &wnd.window_expr[idx - input_field_count]; + match &w.result_type { + Some(t) => { + let expected = to_arrow_datatype(t); + if &expected != field.data_type() { + needs_cast = true; + Arc::new(CastExpr::new(col, expected, None)) + } else { + col + } + } + None => col, + } + } else { + col + }; + proj_exprs.push((expr, field.name().to_string())); + } + if needs_cast { + Arc::new(ProjectionExec::try_new(proj_exprs, window_agg)?) + } else { + window_agg + } + }; + Ok(( scans, shuffle_scans, - Arc::new(SparkPlan::new(spark_plan.plan_id, window_agg, vec![child])), + Arc::new(SparkPlan::new(spark_plan.plan_id, final_plan, vec![child])), )) } OpStruct::ShuffleScan(scan) => { @@ -2444,6 +2513,7 @@ impl PhysicalPlanner { partition_by: &[Arc], sort_exprs: &[PhysicalSortExpr], ) -> Result, ExecutionError> { + let window_func: WindowFunctionDefinition; let window_func_name: String; let window_args: Vec>; if let Some(func) = &spark_expr.built_in_window_function { @@ -2455,6 +2525,13 @@ impl PhysicalPlanner { .iter() .map(|expr| self.create_expr(expr, Arc::clone(&input_schema))) .collect::, ExecutionError>>()?; + window_func = + self.find_df_window_function(&window_func_name) + .ok_or_else(|| { + GeneralError(format!( + "{window_func_name} not supported for window function" + )) + })?; } other => { return Err(GeneralError(format!( @@ -2463,24 +2540,32 @@ impl PhysicalPlanner { } }; } else if let Some(agg_func) = &spark_expr.agg_func { - let result = self.process_agg_func(agg_func, Arc::clone(&input_schema))?; - window_func_name = result.0; - window_args = result.1; + // Is the frame ever-expanding (start = UnboundedPreceding)? When it is, + // DataFusion uses `PlainAggregateWindowExpr` which does not call + // `retract_batch`, so we can safely use Comet's Spark-compatible + // UDAFs (SumDecimal/SumInteger/AvgDecimal/Avg). Otherwise it uses + // `SlidingAggregateWindowExpr` which requires retract — Comet's UDAFs + // don't implement it, so the caller must fall back to DataFusion's + // built-ins (which do). + let is_ever_expanding = spark_expr + .spec + .as_ref() + .and_then(|s| s.frame_specification.as_ref()) + .and_then(|f| f.lower_bound.as_ref()) + .and_then(|lb| lb.lower_frame_bound_struct.as_ref()) + .map(|inner| matches!(inner, LowerFrameBoundStruct::UnboundedPreceding(_))) + .unwrap_or(true); + let (func, args) = + self.process_agg_func(agg_func, Arc::clone(&input_schema), is_ever_expanding)?; + window_func_name = func.name().to_string(); + window_args = args; + window_func = func; } else { return Err(GeneralError( "Both func and agg_func are not set".to_string(), )); } - let window_func = match self.find_df_window_function(&window_func_name) { - Some(f) => f, - _ => { - return Err(GeneralError(format!( - "{window_func_name} not supported for window function" - ))) - } - }; - let spark_window_frame = match spark_expr .spec .as_ref() @@ -2519,13 +2604,26 @@ impl PhysicalPlanner { } }, LowerFrameBoundStruct::Preceding(offset) => { - let offset_value = offset.offset.abs(); + // Spark encodes ROWS bound direction via the sign of the offset: + // negative => PRECEDING, positive => FOLLOWING. The proto + // `LowerWindowFrameBound` only carries a `Preceding` variant, so + // Comet overloads it for both cases. Route to the matching + // DataFusion `WindowFrameBound` based on the sign. match units { - WindowFrameUnits::Rows => WindowFrameBound::Preceding(ScalarValue::UInt64( - Some(offset_value as u64), - )), + WindowFrameUnits::Rows => { + let abs = offset.offset.unsigned_abs(); + if offset.offset < 0 { + WindowFrameBound::Preceding(ScalarValue::UInt64(Some(abs))) + } else { + WindowFrameBound::Following(ScalarValue::UInt64(Some(abs))) + } + } WindowFrameUnits::Range => { - WindowFrameBound::Preceding(ScalarValue::Int64(Some(offset_value))) + let scalar = match offset.range_offset.as_ref() { + Some(lit) => numeric_literal_to_scalar(lit)?, + None => ScalarValue::Int64(Some(offset.offset.abs())), + }; + WindowFrameBound::Preceding(scalar) } WindowFrameUnits::Groups => { return Err(GeneralError( @@ -2568,10 +2666,23 @@ impl PhysicalPlanner { }, UpperFrameBoundStruct::Following(offset) => match units { WindowFrameUnits::Rows => { - WindowFrameBound::Following(ScalarValue::UInt64(Some(offset.offset as u64))) + // Mirror the lower-bound sign handling: the upper proto + // variant is `Following`, but Spark encodes the bound + // direction via sign. Negative => PRECEDING, positive => + // FOLLOWING. + let abs = offset.offset.unsigned_abs(); + if offset.offset < 0 { + WindowFrameBound::Preceding(ScalarValue::UInt64(Some(abs))) + } else { + WindowFrameBound::Following(ScalarValue::UInt64(Some(abs))) + } } WindowFrameUnits::Range => { - WindowFrameBound::Following(ScalarValue::Int64(Some(offset.offset))) + let scalar = match offset.range_offset.as_ref() { + Some(lit) => numeric_literal_to_scalar(lit)?, + None => ScalarValue::Int64(Some(offset.offset)), + }; + WindowFrameBound::Following(scalar) } WindowFrameUnits::Groups => { return Err(GeneralError( @@ -2615,7 +2726,22 @@ impl PhysicalPlanner { &self, agg_func: &AggExpr, schema: SchemaRef, - ) -> Result<(String, Vec>), ExecutionError> { + is_ever_expanding: bool, + ) -> Result<(WindowFunctionDefinition, Vec>), ExecutionError> { + // Wrap a freshly-constructed AggregateUDF impl as a WindowFunctionDefinition. + fn udaf( + udaf: U, + ) -> WindowFunctionDefinition { + WindowFunctionDefinition::AggregateUDF(Arc::new(AggregateUDF::new_from_impl(udaf))) + } + + // Resolve a window-capable function by name via the session registry, returning + // a clean "X not supported for window function" error if missing. + let by_name = |name: &str| -> Result { + self.find_df_window_function(name) + .ok_or_else(|| GeneralError(format!("{name} not supported for window function"))) + }; + match &agg_func.expr_struct { Some(AggExprStruct::Count(expr)) => { let children = expr @@ -2623,27 +2749,98 @@ impl PhysicalPlanner { .iter() .map(|child| self.create_expr(child, Arc::clone(&schema))) .collect::, _>>()?; - Ok(("count".to_string(), children)) + Ok((by_name("count")?, children)) } Some(AggExprStruct::Min(expr)) => { let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?; - Ok(("min".to_string(), vec![child])) + Ok((by_name("min")?, vec![child])) } Some(AggExprStruct::Max(expr)) => { let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?; - Ok(("max".to_string(), vec![child])) + Ok((by_name("max")?, vec![child])) } Some(AggExprStruct::Sum(expr)) => { + // For ever-expanding frames, use Comet's Spark-compatible Sum UDAFs + // (SumDecimal / SumInteger) which enforce Spark overflow semantics. + // For sliding frames, those UDAFs can't be used (no retract_batch), + // so delegate to DataFusion's built-in `sum`, which supports retract + // but doesn't enforce Spark's decimal precision overflow-to-NULL. let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?; let arrow_type = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - let datatype = child.data_type(&schema)?; - - let child = if datatype != arrow_type { - Arc::new(CastExpr::new(child, arrow_type.clone(), None)) - } else { - child - }; - Ok(("sum".to_string(), vec![child])) + match arrow_type { + DataType::Decimal128(_, _) if is_ever_expanding => { + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; + let func = SumDecimal::try_new( + arrow_type, + eval_mode, + agg_func.expr_id, + Arc::clone(&self.query_context_registry), + )?; + Ok((udaf(func), vec![child])) + } + DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 + if is_ever_expanding => + { + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; + let func = SumInteger::try_new(arrow_type, eval_mode)?; + Ok((udaf(func), vec![child])) + } + _ => { + let actual = child.data_type(&schema)?; + let child: Arc = if actual != arrow_type { + Arc::new(CastExpr::new(child, arrow_type, None)) + } else { + child + }; + Ok((by_name("sum")?, vec![child])) + } + } + } + Some(AggExprStruct::Avg(expr)) => { + // Same rule as Sum: Comet's Avg/AvgDecimal for ever-expanding frames, + // DataFusion's `avg` for sliding (retract-capable). + let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + let input_datatype = to_arrow_datatype(expr.sum_datatype.as_ref().unwrap()); + match datatype { + DataType::Decimal128(_, _) if is_ever_expanding => { + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; + let func = AvgDecimal::new( + datatype, + input_datatype, + eval_mode, + agg_func.expr_id, + Arc::clone(&self.query_context_registry), + ); + Ok((udaf(func), vec![child])) + } + _ if is_ever_expanding => { + let child: Arc = + Arc::new(CastExpr::new(child, DataType::Float64, None)); + let func = Avg::new("avg", DataType::Float64); + Ok((udaf(func), vec![child])) + } + _ => { + // Sliding frame — DataFusion's built-in `avg` handles retract. + // Cast non-decimal input to Float64 to match Spark's Avg result type. + let child: Arc = match datatype { + DataType::Decimal128(_, _) => child, + _ => Arc::new(CastExpr::new(child, DataType::Float64, None)), + }; + Ok((by_name("avg")?, vec![child])) + } + } + } + Some(AggExprStruct::First(expr)) => { + // Spark's FIRST_VALUE → DataFusion's `first_value` UDAF. The UDAF honors + // ignore-nulls via the WindowExpr-level `ignore_nulls` flag. + let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?; + Ok((by_name("first_value")?, vec![child])) + } + Some(AggExprStruct::Last(expr)) => { + // Spark's LAST_VALUE → DataFusion's `last_value` UDAF. + let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?; + Ok((by_name("last_value")?, vec![child])) } other => Err(GeneralError(format!( "{other:?} not supported for window function" @@ -2654,14 +2851,20 @@ impl PhysicalPlanner { /// Find DataFusion's built-in window function by name. fn find_df_window_function(&self, name: &str) -> Option { let registry = &self.session_ctx.state(); + // Prefer the WindowUDF (frame-aware) over the AggregateUDF when both exist. + // Functions like `nth_value` are registered twice in DataFusion: once as an + // aggregate and once as a window function. The window variant has the + // correct IGNORE NULLS / frame semantics for an OVER clause; the aggregate + // variant wrapped into PlainAggregateWindowExpr does not. Aggregate-only + // functions (count/sum/min/max/avg/...) have no UDWF and fall through. registry - .udaf(name) - .map(WindowFunctionDefinition::AggregateUDF) + .udwf(name) + .map(WindowFunctionDefinition::WindowUDF) .ok() .or_else(|| { registry - .udwf(name) - .map(WindowFunctionDefinition::WindowUDF) + .udaf(name) + .map(WindowFunctionDefinition::AggregateUDF) .ok() }) } @@ -2961,6 +3164,62 @@ fn expr_to_columns( Ok((left_field_indices, right_field_indices)) } +/// Convert a Spark numeric Literal proto into a `ScalarValue` whose data type +/// matches the literal's declared type. Used for RANGE window frame offsets, +/// where the offset's type must match the ORDER BY column's type. Only numeric +/// types are supported; the Scala side rejects non-numeric RANGE offsets before +/// reaching here. +fn numeric_literal_to_scalar( + lit: &spark_expression::Literal, +) -> Result { + let data_type = to_arrow_datatype(lit.datatype.as_ref().ok_or_else(|| { + GeneralError("RANGE frame offset literal is missing datatype".to_string()) + })?); + + if lit.is_null { + return Err(GeneralError( + "RANGE frame offset must not be null".to_string(), + )); + } + + let value = lit + .value + .as_ref() + .ok_or_else(|| GeneralError("RANGE frame offset literal has no value".to_string()))?; + + let scalar = match value { + Value::ByteVal(v) => ScalarValue::Int8(Some(*v as i8)), + Value::ShortVal(v) => ScalarValue::Int16(Some(*v as i16)), + Value::IntVal(v) => ScalarValue::Int32(Some(*v)), + Value::LongVal(v) => ScalarValue::Int64(Some(*v)), + Value::FloatVal(v) => ScalarValue::Float32(Some(*v)), + Value::DoubleVal(v) => ScalarValue::Float64(Some(*v)), + Value::DecimalVal(bytes) => { + let big_integer = BigInt::from_signed_bytes_be(bytes); + let integer = big_integer.to_i128().ok_or_else(|| { + GeneralError(format!( + "Cannot parse {big_integer:?} as i128 for Decimal RANGE frame offset" + )) + })?; + match data_type { + DataType::Decimal128(p, s) => ScalarValue::Decimal128(Some(integer), p, s), + ref dt => { + return Err(GeneralError(format!( + "Decimal RANGE frame offset has non-Decimal128 datatype: {dt:?}" + ))) + } + } + } + other => { + return Err(GeneralError(format!( + "Unsupported value variant for RANGE frame offset: {other:?}" + ))) + } + }; + + Ok(scalar) +} + /// A physical join filter rewritter which rewrites the column indices in the expression /// to use the new column indices. See `rewrite_physical_expr`. struct JoinFilterRewriter<'a> { diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 7cefe06da7..8b33d63971 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -22,6 +22,7 @@ syntax = "proto3"; package spark.spark_operator; import "expr.proto"; +import "literal.proto"; import "partitioning.proto"; import "types.proto"; @@ -395,6 +396,10 @@ message WindowExpr { spark.spark_expression.AggExpr agg_func = 2; WindowSpecDefinition spec = 3; bool ignore_nulls = 4; + // Spark's expected result type. Used to cast the native window function output + // when DataFusion's return type differs (e.g. row_number returns UInt64 but + // Spark expects Int32). + spark.spark_expression.DataType result_type = 5; } enum WindowFrameType { @@ -425,11 +430,19 @@ message UpperWindowFrameBound { } message Preceding { + // Used for ROWS frames. Integer row count. int64 offset = 1; + // Used for RANGE frames. Carries the typed offset value so the native + // side can build a ScalarValue whose type matches the ORDER BY column. + spark.spark_expression.Literal range_offset = 2; } message Following { + // Used for ROWS frames. Integer row count. int64 offset = 1; + // Used for RANGE frames. Carries the typed offset value so the native + // side can build a ScalarValue whose type matches the ORDER BY column. + spark.spark_expression.Literal range_offset = 2; } message UnboundedPreceding {} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index e642bafa4f..37950aa044 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -21,67 +21,43 @@ package org.apache.spark.sql.comet import scala.jdk.CollectionConverters._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, CurrentRow, Expression, FrameLessOffsetWindowFunction, Lag, Lead, NamedExpression, RangeFrame, RowFrame, SortOrder, SpecifiedWindowFrame, UnboundedFollowing, UnboundedPreceding, WindowExpression} -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count, Max, Min, Sum} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeSet, CumeDist, CurrentRow, DenseRank, Expression, Lag, Lead, Literal, MakeDecimal, NamedExpression, NthValue, NTile, PercentRank, RangeFrame, Rank, RowFrame, RowNumber, SortOrder, SpecifiedWindowFrame, UnboundedFollowing, UnboundedPreceding, WindowExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, Complete, Count, First, Last, Max, Min, Sum} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.{LongType, NumericType} +import org.apache.spark.sql.types.Decimal import com.google.common.base.Objects import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo -import org.apache.comet.serde.{AggSerde, CometOperatorSerde, Incompatible, OperatorOuterClass, SupportLevel} +import org.apache.comet.serde.{AggSerde, CometOperatorSerde, LiteralOuterClass, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator -import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto, scalarFunctionExprToProto} +import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto, scalarFunctionExprToProto, serializeDataType} object CometWindowExec extends CometOperatorSerde[WindowExec] { override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( CometConf.COMET_EXEC_WINDOW_ENABLED) - override def getSupportLevel(op: WindowExec): SupportLevel = { - Incompatible(Some("Native WindowExec has known correctness issues")) - } - override def convert( op: WindowExec, builder: Operator.Builder, childOp: OperatorOuterClass.Operator*): Option[OperatorOuterClass.Operator] = { val output = op.child.output - val winExprs: Array[WindowExpression] = op.windowExpression.flatMap { expr => - expr match { - case alias: Alias => - alias.child match { - case winExpr: WindowExpression => - Some(winExpr) - case _ => - None - } - case _ => - None - } + val winExprs: Array[WindowExpression] = op.windowExpression.map { + case Alias(w: WindowExpression, _) => w + case Alias(MakeDecimal(w: WindowExpression, _, _, _), _) => w + case other => + withInfo(op, s"Unsupported window expression: $other", other) + return None }.toArray - if (winExprs.length != op.windowExpression.length) { - withInfo(op, "Unsupported window expression(s)") - return None - } - - // Offset window functions (LAG, LEAD) support arbitrary partition and order specs, so skip - // the validatePartitionAndSortSpecsForWindowFunc check which requires partition columns to - // equal order columns. That stricter check is only needed for aggregate window functions. - val hasOnlyOffsetFunctions = winExprs.nonEmpty && - winExprs.forall(e => e.windowFunction.isInstanceOf[FrameLessOffsetWindowFunction]) - if (!hasOnlyOffsetFunctions && op.partitionSpec.nonEmpty && op.orderSpec.nonEmpty && - !validatePartitionAndSortSpecsForWindowFunc(op.partitionSpec, op.orderSpec, op)) { - return None - } - val windowExprProto = winExprs.map(windowExprToProto(_, output, op.conf)) val partitionExprs = op.partitionSpec.map(exprToProto(_, op.child.output)) @@ -95,9 +71,16 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { windowBuilder.addAllOrderByList(sortOrders.map(_.get).asJava) Some(builder.setWindow(windowBuilder).build()) } else { + // Roll up reasons already attached to per-expression nodes so the Window + // operator itself carries a fallback attribution. Without this, the plan + // prints a bare `Window` and the real reason lives on a sub-expression + // that isn't obvious in the standard explain output. + val failing = winExprs.toSeq.zip(windowExprProto).collect { case (we, None) => we } ++ + op.partitionSpec.zip(partitionExprs).collect { case (e, None) => e } ++ + op.orderSpec.zip(sortOrders).collect { case (e, None) => e } + withInfo(op, failing: _*) None } - } private def windowExprToProto( @@ -126,13 +109,23 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { None } case s: Sum => - if (AggSerde.sumDataTypeSupported(s.dataType) && !s.dataType - .isInstanceOf[DecimalType]) { + if (AggSerde.sumDataTypeSupported(s.dataType)) { Some(agg) } else { withInfo(windowExpr, s"datatype ${s.dataType} is not supported", expr) None } + case a: Average => + if (AggSerde.avgDataTypeSupported(a.dataType)) { + Some(agg) + } else { + withInfo(windowExpr, s"datatype ${a.dataType} is not supported", expr) + None + } + case _: First => + Some(agg) + case _: Last => + Some(agg) case _ => withInfo( windowExpr, @@ -146,26 +139,83 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { } }.toArray + // If the window function is itself an (unsupported) AggregateExpression the + // filter above already recorded a specific reason on `windowExpr`. Short-circuit + // here to avoid the fallthrough `exprToProto` path tagging an additional generic + // "aggregateexpression is not supported" message. + if (aggregateExpressions.isEmpty && + windowExpr.windowFunction.isInstanceOf[AggregateExpression]) { + return None + } + val (aggExpr, builtinFunc, ignoreNulls) = if (aggregateExpressions.nonEmpty) { val modes = aggregateExpressions.map(_.mode).distinct assert(modes.size == 1 && modes.head == Complete) - (aggExprToProto(aggregateExpressions.head, output, true, conf), None, false) + val agg = aggregateExpressions.head + val ignoreNulls = agg.aggregateFunction match { + case f: First => f.ignoreNulls + case l: Last => l.ignoreNulls + case _ => false + } + (aggExprToProto(agg, output, true, conf), None, ignoreNulls) } else { windowExpr.windowFunction match { + case lag: Lag if !lag.default.isInstanceOf[Literal] => + // https://github.com/apache/datafusion-comet/issues/4268 + withInfo(windowExpr, "Lag default value must be a literal", lag.default) + (None, None, false) case lag: Lag => val inputExpr = exprToProto(lag.input, output) val offsetExpr = exprToProto(lag.inputOffset, output) val defaultExpr = exprToProto(lag.default, output) val func = scalarFunctionExprToProto("lag", inputExpr, offsetExpr, defaultExpr) (None, func, lag.ignoreNulls) + case lead: Lead if !lead.default.isInstanceOf[Literal] => + // https://github.com/apache/datafusion-comet/issues/4268 + withInfo(windowExpr, "Lead default value must be a literal", lead.default) + (None, None, false) case lead: Lead => val inputExpr = exprToProto(lead.input, output) val offsetExpr = exprToProto(lead.offset, output) val defaultExpr = exprToProto(lead.default, output) val func = scalarFunctionExprToProto("lead", inputExpr, offsetExpr, defaultExpr) (None, func, lead.ignoreNulls) - case _ => - (None, exprToProto(windowExpr.windowFunction, output), false) + case _: RowNumber => + (None, scalarFunctionExprToProto("row_number"), false) + case _: Rank => + (None, scalarFunctionExprToProto("rank"), false) + case _: DenseRank => + (None, scalarFunctionExprToProto("dense_rank"), false) + case _: PercentRank => + (None, scalarFunctionExprToProto("percent_rank"), false) + case _: CumeDist => + (None, scalarFunctionExprToProto("cume_dist"), false) + case nt: NTile => + // Known correctness bug: Comet's NTILE produces different bucket + // assignments than Spark; tracked in #4255. Fall back to Spark. + withInfo(windowExpr, "NTILE has a correctness bug in Comet tracked in #4255", nt) + (None, None, false) + case nv: NthValue => + val inputExpr = exprToProto(nv.input, output) + + val offsetExpr = nv.offset.eval() match { + case n: Number => + exprToProto(Literal(n.longValue(), LongType), output) + case _ => + withInfo( + windowExpr, + s"Unsupported NTH_VALUE offset: ${nv.offset} (${nv.offset.dataType})") + None + } + val func = scalarFunctionExprToProto("nth_value", inputExpr, offsetExpr) + (None, func, nv.ignoreNulls) + + case other => + withInfo( + windowExpr, + s"window function ${other.getClass.getSimpleName} is not supported", + other) + (None, None, false) } } @@ -197,7 +247,9 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { val offset = e.eval() match { case i: Integer => i.toLong case l: Long => l - case _ => return None + case _ => + withInfo(windowExpr, s"Unsupported ROWS frame lower offset: $e (${e.dataType})") + return None } OperatorOuterClass.LowerWindowFrameBound .newBuilder() @@ -207,9 +259,25 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { .setOffset(offset) .build()) .build() - case _ => - // TODO add support for numeric and temporal RANGE BETWEEN expressions - // see https://github.com/apache/datafusion-comet/issues/1246 + case e if frameType == RangeFrame && e.dataType.isInstanceOf[NumericType] => + rangeBoundLiteral(e, isLower = true, output) match { + case Some(lit) => + OperatorOuterClass.LowerWindowFrameBound + .newBuilder() + .setPreceding( + OperatorOuterClass.Preceding + .newBuilder() + .setRangeOffset(lit) + .build()) + .build() + case None => + withInfo(windowExpr, s"Unsupported RANGE frame lower offset: $e") + return None + } + case e => + withInfo( + windowExpr, + s"RANGE frame with non-numeric offset is not supported: ${e.dataType}") return None } @@ -228,7 +296,9 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { val offset = e.eval() match { case i: Integer => i.toLong case l: Long => l - case _ => return None + case _ => + withInfo(windowExpr, s"Unsupported ROWS frame upper offset: $e (${e.dataType})") + return None } OperatorOuterClass.UpperWindowFrameBound .newBuilder() @@ -238,9 +308,25 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { .setOffset(offset) .build()) .build() - case _ => - // TODO add support for numeric and temporal RANGE BETWEEN expressions - // see https://github.com/apache/datafusion-comet/issues/1246 + case e if frameType == RangeFrame && e.dataType.isInstanceOf[NumericType] => + rangeBoundLiteral(e, isLower = false, output) match { + case Some(lit) => + OperatorOuterClass.UpperWindowFrameBound + .newBuilder() + .setFollowing( + OperatorOuterClass.Following + .newBuilder() + .setRangeOffset(lit) + .build()) + .build() + case None => + withInfo(windowExpr, s"Unsupported RANGE frame upper offset: $e") + return None + } + case e => + withInfo( + windowExpr, + s"RANGE frame with non-numeric offset is not supported: ${e.dataType}") return None } @@ -268,21 +354,24 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { val spec = OperatorOuterClass.WindowSpecDefinition.newBuilder().setFrameSpecification(frame).build() + val resultTypeProto = serializeDataType(windowExpr.dataType) + if (builtinFunc.isDefined) { - Some( - OperatorOuterClass.WindowExpr - .newBuilder() - .setBuiltInWindowFunction(builtinFunc.get) - .setSpec(spec) - .setIgnoreNulls(ignoreNulls) - .build()) + val b = OperatorOuterClass.WindowExpr + .newBuilder() + .setBuiltInWindowFunction(builtinFunc.get) + .setSpec(spec) + .setIgnoreNulls(ignoreNulls) + resultTypeProto.foreach(b.setResultType) + Some(b.build()) } else if (aggExpr.isDefined) { - Some( - OperatorOuterClass.WindowExpr - .newBuilder() - .setAggFunc(aggExpr.get) - .setSpec(spec) - .build()) + val b = OperatorOuterClass.WindowExpr + .newBuilder() + .setAggFunc(aggExpr.get) + .setSpec(spec) + .setIgnoreNulls(ignoreNulls) + resultTypeProto.foreach(b.setResultType) + Some(b.build()) } else { None } @@ -300,38 +389,56 @@ object CometWindowExec extends CometOperatorSerde[WindowExec] { SerializedPlan(None)) } - private def validatePartitionAndSortSpecsForWindowFunc( - partitionSpec: Seq[Expression], - orderSpec: Seq[SortOrder], - op: SparkPlan): Boolean = { - if (partitionSpec.length != orderSpec.length) { - return false - } - - val partitionColumnNames = partitionSpec.collect { - case a: AttributeReference => a.name - case other => - withInfo(op, s"Unsupported partition expression: ${other.getClass.getSimpleName}") - return false - } - - val orderColumnNames = orderSpec.collect { case s: SortOrder => - s.child match { - case a: AttributeReference => a.name - case other => - withInfo(op, s"Unsupported sort expression: ${other.getClass.getSimpleName}") - return false + // Folds a RANGE frame bound expression to a constant and serializes its + // magnitude as a typed Literal proto. Spark encodes PRECEDING/FOLLOWING via + // the sign of the literal (negative => PRECEDING, positive => FOLLOWING), + // but the proto only carries magnitude with direction implied by Lower vs + // Upper position. So we reject lower=positive (FOLLOWING) and upper=negative + // (PRECEDING) by returning None. + private def rangeBoundLiteral( + bound: Expression, + isLower: Boolean, + output: Seq[Attribute]): Option[LiteralOuterClass.Literal] = { + val rawValue = + try { + bound.eval() + } catch { + case _: Exception => return None } + if (rawValue == null) { + return None } - - if (partitionColumnNames.zip(orderColumnNames).exists { case (partCol, orderCol) => - partCol != orderCol - }) { - withInfo(op, "Partitioning and sorting specifications must be the same.") - return false + // Taking the absolute value of the narrow signed MIN_VALUE constants + // overflows silently (Math.abs(Byte.MinValue).toByte == Byte.MinValue), + // which would flip the sign and produce an incorrect frame bound. Reject + // those pathological inputs up front. + val (signum, absValue): (Int, Any) = rawValue match { + case b: java.lang.Byte => + if (b.byteValue() == Byte.MinValue) return None + (Integer.signum(b.intValue()), java.lang.Byte.valueOf(Math.abs(b.intValue()).toByte)) + case s: java.lang.Short => + if (s.shortValue() == Short.MinValue) return None + (Integer.signum(s.intValue()), java.lang.Short.valueOf(Math.abs(s.intValue()).toShort)) + case i: java.lang.Integer => + if (i.intValue() == Int.MinValue) return None + (Integer.signum(i.intValue()), java.lang.Integer.valueOf(Math.abs(i.intValue()))) + case l: java.lang.Long => + if (l.longValue() == Long.MinValue) return None + (java.lang.Long.signum(l.longValue()), java.lang.Long.valueOf(Math.abs(l.longValue()))) + case f: java.lang.Float => + (Math.signum(f.doubleValue()).toInt, java.lang.Float.valueOf(Math.abs(f.floatValue()))) + case d: java.lang.Double => + (Math.signum(d.doubleValue()).toInt, java.lang.Double.valueOf(Math.abs(d.doubleValue()))) + case d: Decimal => + (d.toBigDecimal.signum, d.abs) + case _ => return None } + if (isLower && signum > 0) return None + if (!isLower && signum < 0) return None - true + exprToProto(Literal(absValue, bound.dataType), output).flatMap { exprProto => + if (exprProto.hasLiteral) Some(exprProto.getLiteral) else None + } } } diff --git a/spark/src/test/resources/sql-tests/windows/window_functions.sql b/spark/src/test/resources/sql-tests/windows/window_functions.sql new file mode 100644 index 0000000000..3275c387bb --- /dev/null +++ b/spark/src/test/resources/sql-tests/windows/window_functions.sql @@ -0,0 +1,652 @@ +-- 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. + +-- ============================================================ +-- Setup: shared tables +-- ============================================================ + +statement +CREATE TABLE emp(dept string, id int, salary int, hire_yr int) USING parquet + +statement +INSERT INTO emp VALUES + ('eng', 1, 100, 2020), + ('eng', 2, 100, 2021), + ('eng', 2, 100, 2021), + ('eng', 3, 150, 2022), + ('eng', 4, NULL, 2023), + ('eng', 5, 200, 2024), + ('sales', 6, 90, 2020), + ('sales', 7, 90, 2021), + ('sales', 8, NULL, 2022), + ('sales', 9, 110, 2023), + ('sales', 10, 120, 2024), + ('ops', 11, NULL, 2020), + ('ops', 11, NULL, 2020), + ('ops', 12, NULL, 2021), + ('ops', 13, 50, 2022) + +statement +CREATE TABLE emp_allnull(dept string, id int, salary int) USING parquet + +statement +INSERT INTO emp_allnull VALUES + ('a', 1, NULL), + ('a', 2, NULL), + ('b', 3, NULL), + ('b', 4, NULL) + +statement +CREATE TABLE daily(dt string, amt int) USING parquet + +statement +INSERT INTO daily VALUES + ('2024-01-01', 10), + ('2024-01-02', 20), + ('2024-01-03', 30), + ('2024-01-04', 40) + +statement +CREATE TABLE scores(player string, game int, score int) USING parquet + +statement +INSERT INTO scores VALUES + ('alice', 1, 10), + ('alice', 2, 10), + ('alice', 3, 20), + ('alice', 4, 30), + ('bob', 1, 10), + ('bob', 2, 10), + ('bob', 3, 10), + ('bob', 4, 40), + ('carol', 1, 50), + ('carol', 2, 50), + ('carol', 3, 50), + ('carol', 4, 50) + +-- ############################################################ +-- Section 1: Basic window combinations +-- ############################################################ + +-- ============================================================ +-- 1.1: aggregate over unbounded window (no ORDER BY) +-- ============================================================ + +query +SELECT id, dept, salary, + COUNT(*) OVER () AS c, + SUM(salary) OVER () AS s, + MAX(salary) OVER () AS mx, + MIN(salary) OVER () AS mn +FROM emp + +-- ============================================================ +-- 1.2: aggregate with ORDER BY (running aggregates) +-- ============================================================ + +query +SELECT id, dept, salary, + COUNT(*) OVER (ORDER BY id) AS c, + SUM(salary) OVER (ORDER BY id) AS s, + MAX(salary) OVER (ORDER BY id) AS mx, + MIN(salary) OVER (ORDER BY id) AS mn +FROM emp + +-- ============================================================ +-- 1.3: aggregate with PARTITION BY + ORDER BY +-- ============================================================ + +query +SELECT dept, id, salary, + COUNT(*) OVER (PARTITION BY dept ORDER BY id) AS c, + SUM(salary) OVER (PARTITION BY dept ORDER BY id) AS s, + AVG(salary) OVER (PARTITION BY dept ORDER BY id) AS a +FROM emp + +-- ============================================================ +-- 1.4: aggregate with ORDER BY DESC +-- ============================================================ + +query +SELECT id, dept, salary, + SUM(salary) OVER (PARTITION BY dept ORDER BY id DESC) AS s +FROM emp + +-- ============================================================ +-- 1.5: ranking functions (ROW_NUMBER, RANK, DENSE_RANK) +-- ============================================================ + +query +SELECT dept, id, salary, + ROW_NUMBER() OVER (PARTITION BY dept ORDER BY id) AS rn, + RANK() OVER (PARTITION BY dept ORDER BY id) AS rk, + DENSE_RANK() OVER (PARTITION BY dept ORDER BY id) AS drk +FROM emp + +-- ============================================================ +-- 1.6: PERCENT_RANK and CUME_DIST +-- ============================================================ + +query tolerance=1e-6 +SELECT dept, id, salary, + PERCENT_RANK() OVER (PARTITION BY dept ORDER BY id) AS pr, + CUME_DIST() OVER (PARTITION BY dept ORDER BY id) AS cd +FROM emp + +-- ============================================================ +-- 1.7: LAG / LEAD with default offset +-- ============================================================ + +query +SELECT dept, id, salary, + LAG(salary) OVER (PARTITION BY dept ORDER BY id) AS lg, + LEAD(salary) OVER (PARTITION BY dept ORDER BY id) AS ld +FROM emp + +-- ============================================================ +-- 1.8: LAG / LEAD with explicit offset and default value +-- ============================================================ + +query +SELECT dept, id, salary, + LAG(salary, 2, -1) OVER (PARTITION BY dept ORDER BY id) AS lg2, + LEAD(salary, 2, -1) OVER (PARTITION BY dept ORDER BY id) AS ld2 +FROM emp + +-- ============================================================ +-- 1.9: ROWS BETWEEN frames +-- Note: Spark parses `N PRECEDING` in ROWS frames as UnaryMinus(Literal(N)), +-- which only becomes a plain Literal(-N) after ConstantFolding runs. +-- CometSqlFileTestSuite disables ConstantFolding, so the ROWS frames below +-- only use bounds that parse directly (UNBOUNDED / CURRENT ROW / N FOLLOWING). +-- N PRECEDING in ROWS frames is covered by CometWindowExecSuite via the +-- DataFrame API, which emits Literal(-N) without needing the optimizer. +-- ============================================================ + +query +SELECT dept, id, salary, + SUM(salary) OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS s_run, + SUM(salary) OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) AS s_lookahead, + SUM(salary) OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING) AS s_window3, + SUM(salary) OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) AS s_tail +FROM emp + +-- ============================================================ +-- 1.10: RANGE BETWEEN with numeric ORDER BY +-- ============================================================ + +query +SELECT dept, id, salary, + SUM(salary) OVER (PARTITION BY dept ORDER BY id + RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS s_rng +FROM emp + +-- ============================================================ +-- 1.11: multiple PARTITION BY + multiple ORDER BY +-- ============================================================ + +query +SELECT dept, id, hire_yr, salary, + SUM(salary) OVER (PARTITION BY dept, hire_yr ORDER BY id) AS s1, + RANK() OVER (PARTITION BY dept ORDER BY hire_yr, id) AS rk +FROM emp + +-- ============================================================ +-- 1.12: complex expression in aggregate input +-- ============================================================ + +query +SELECT dept, id, salary, + SUM(salary + 10) OVER (PARTITION BY dept ORDER BY id) AS s_plus_10 +FROM emp + +-- ============================================================ +-- 1.13: multiple window functions with mixed specs in one query +-- ============================================================ + +query +SELECT dept, id, salary, + ROW_NUMBER() OVER (PARTITION BY dept ORDER BY id) AS rn, + RANK() OVER (PARTITION BY dept ORDER BY salary) AS rk_salary, + SUM(salary) OVER (PARTITION BY dept ORDER BY id) AS run_sum, + MAX(salary) OVER () AS global_max +FROM emp + +-- ############################################################ +-- Section 2: Mixed nulls +-- ############################################################ + +-- ============================================================ +-- 2.1: COUNT(*) vs COUNT(col) with nulls +-- ============================================================ + +query +SELECT dept, id, salary, + COUNT(*) OVER (PARTITION BY dept ORDER BY id) AS c_star, + COUNT(salary) OVER (PARTITION BY dept ORDER BY id) AS c_col +FROM emp + +-- ============================================================ +-- 2.2: SUM / AVG / MIN / MAX skip nulls +-- ============================================================ + +query +SELECT dept, id, salary, + SUM(salary) OVER (PARTITION BY dept ORDER BY id) AS s, + AVG(salary) OVER (PARTITION BY dept ORDER BY id) AS a, + MIN(salary) OVER (PARTITION BY dept ORDER BY id) AS mn, + MAX(salary) OVER (PARTITION BY dept ORDER BY id) AS mx +FROM emp + +-- ============================================================ +-- 2.3: LAG / LEAD RESPECT NULLS (default) vs IGNORE NULLS +-- ============================================================ + +query +SELECT dept, id, salary, + LAG(salary) OVER (PARTITION BY dept ORDER BY id) AS lag_respect, + LAG(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id) AS lag_ignore, + LEAD(salary) OVER (PARTITION BY dept ORDER BY id) AS lead_respect, + LEAD(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id) AS lead_ignore +FROM emp + +-- ============================================================ +-- 2.4: FIRST_VALUE / LAST_VALUE RESPECT vs IGNORE NULLS +-- ============================================================ +query +SELECT dept, id, salary, + FIRST_VALUE(salary) OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS fv_respect, + FIRST_VALUE(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS fv_ignore, + LAST_VALUE(salary) OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS lv_respect, + LAST_VALUE(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS lv_ignore +FROM emp + +-- ============================================================ +-- 2.5: all-null partition +-- ============================================================ +query +SELECT dept, id, salary, + SUM(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS s, + AVG(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS a, + COUNT(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS c_col, + FIRST_VALUE(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS fv_ig +FROM emp_allnull + +-- ============================================================ +-- 2.6: NTH_VALUE with nulls RESPECTed vs IGNORED +-- ============================================================ + +query +SELECT dept, id, salary, + NTH_VALUE(salary, 2) OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS nth_respect, + NTH_VALUE(salary, 2) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS nth_ignore +FROM emp + +query +SELECT dept, id, salary, + first_value(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS fv_ignore, + last_value(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS lv_ignore, + nth_value(salary, 1) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS nth_ignore +FROM emp + +-- ############################################################ +-- Section 3: Window functions with GROUP BY +-- ############################################################ + +-- ============================================================ +-- 3.1: window over aggregated rows (window in outer SELECT) +-- ============================================================ + +query +SELECT dept, total_salary, + RANK() OVER (ORDER BY total_salary DESC) AS rk +FROM ( + SELECT dept, SUM(salary) AS total_salary + FROM emp + GROUP BY dept +) t + +-- ============================================================ +-- 3.2: window and GROUP BY in the same query (aggregated input to window) +-- ============================================================ + +query +SELECT dept, + SUM(salary) AS dept_total, + SUM(SUM(salary)) OVER () AS grand_total, + SUM(salary) / SUM(SUM(salary)) OVER () AS share +FROM emp +GROUP BY dept + +-- ============================================================ +-- 3.3: window with GROUP BY over multiple keys +-- ============================================================ + +query +SELECT dept, hire_yr, + SUM(salary) AS yr_total, + SUM(SUM(salary)) OVER (PARTITION BY dept ORDER BY hire_yr) AS dept_running +FROM emp +WHERE salary IS NOT NULL +GROUP BY dept, hire_yr + +-- ============================================================ +-- 3.4: GROUP BY producing a single row per group, RANK compares groups +-- ============================================================ + +query +SELECT dept, cnt, avg_sal, + DENSE_RANK() OVER (ORDER BY avg_sal DESC) AS rk_by_avg +FROM ( + SELECT dept, COUNT(*) AS cnt, AVG(salary) AS avg_sal + FROM emp + GROUP BY dept +) t + +-- ############################################################ +-- Section 4: LAST_VALUE default frame semantics +-- ############################################################ +-- SQL default frame when ORDER BY is present is +-- RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +-- so LAST_VALUE without an explicit frame returns the current row's value, +-- NOT the last row of the partition. Most users intuitively expect the +-- latter; the explicit ROWS BETWEEN ... UNBOUNDED FOLLOWING form gives it. + +-- ============================================================ +-- 4.1: LAST_VALUE default frame == current row value +-- ============================================================ + +query +SELECT dt, amt, + LAST_VALUE(amt) OVER (ORDER BY dt) AS lv_default +FROM daily + +-- ============================================================ +-- 4.2: LAST_VALUE explicit ROWS BETWEEN ... UNBOUNDED FOLLOWING +-- ============================================================ + +query +SELECT dt, amt, + LAST_VALUE(amt) OVER (ORDER BY dt + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS lv_full +FROM daily + +-- ============================================================ +-- 4.3: contrast default vs explicit FIRST_VALUE / LAST_VALUE +-- ============================================================ + +query +SELECT dt, amt, + FIRST_VALUE(amt) OVER (ORDER BY dt) AS fv_default, + FIRST_VALUE(amt) OVER (ORDER BY dt + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS fv_full, + LAST_VALUE(amt) OVER (ORDER BY dt) AS lv_default, + LAST_VALUE(amt) OVER (ORDER BY dt + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS lv_full +FROM daily + +-- ============================================================ +-- 4.4: LAST_VALUE default with PARTITION BY +-- ============================================================ + +query +SELECT dept, id, salary, + LAST_VALUE(salary) OVER (PARTITION BY dept ORDER BY id) AS lv_default, + LAST_VALUE(salary) OVER (PARTITION BY dept ORDER BY id + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS lv_full +FROM emp + +-- ============================================================ +-- 4.5: LAST_VALUE default RANGE semantics over peer groups +-- Peer rows (same ORDER BY key) see each other's value in the default +-- RANGE frame, so LAST_VALUE within a peer group returns the last peer's +-- value rather than just the current row. Returning `score` (the ORDER BY +-- column) keeps the output deterministic since all peers in a peer group +-- share the same score by definition. +-- ============================================================ + +query +SELECT player, game, score, + LAST_VALUE(score) OVER (PARTITION BY player ORDER BY score) AS lv_default, + FIRST_VALUE(score) OVER (PARTITION BY player ORDER BY score) AS fv_default +FROM scores + +-- ############################################################ +-- Section 5: Peer groups with duplicates +-- ############################################################ +-- Peer groups = rows that share the same ORDER BY key within a partition. +-- RANK assigns the same rank to peers and then skips (1,1,3). DENSE_RANK +-- assigns the same rank to peers without skipping (1,1,2). ROW_NUMBER +-- breaks ties arbitrarily but deterministically within a single plan. + +-- ============================================================ +-- 5.1: RANK vs DENSE_RANK vs ROW_NUMBER with ties +-- ROW_NUMBER needs a tie-breaker (game) to be deterministic — tied rows +-- can get row-numbers in either order otherwise. RANK and DENSE_RANK +-- give peers the same rank regardless of peer order, so they stay on +-- the tied `ORDER BY score` to demonstrate tie handling. +-- ============================================================ + +query +SELECT player, game, score, + ROW_NUMBER() OVER (PARTITION BY player ORDER BY score, game) AS rn, + RANK() OVER (PARTITION BY player ORDER BY score) AS rk, + DENSE_RANK() OVER (PARTITION BY player ORDER BY score) AS drk +FROM scores + +-- ============================================================ +-- 5.2: PERCENT_RANK and CUME_DIST across peer groups +-- CUME_DIST returns (#rows with order_by <= current) / (#rows in partition), +-- so every peer gets the same CUME_DIST value. PERCENT_RANK uses (rank-1)/(n-1), +-- so peers share a PERCENT_RANK as well. +-- ============================================================ + +query tolerance=1e-6 +SELECT player, game, score, + PERCENT_RANK() OVER (PARTITION BY player ORDER BY score) AS pr, + CUME_DIST() OVER (PARTITION BY player ORDER BY score) AS cd +FROM scores + +-- ============================================================ +-- 5.3: peer group where all rows share the same ORDER BY key +-- Every row is a peer of every other, so RANK / DENSE_RANK = 1 for all +-- and CUME_DIST = 1 for all. +-- ============================================================ + +query tolerance=1e-6 +SELECT player, game, score, + RANK() OVER (PARTITION BY player ORDER BY score) AS rk, + DENSE_RANK() OVER (PARTITION BY player ORDER BY score) AS drk, + CUME_DIST() OVER (PARTITION BY player ORDER BY score) AS cd, + PERCENT_RANK() OVER (PARTITION BY player ORDER BY score) AS pr +FROM scores +WHERE player = 'carol' + +-- ============================================================ +-- 5.4: SUM / COUNT default RANGE frame over peer groups +-- Default frame (RANGE UNBOUNDED PRECEDING TO CURRENT ROW) includes all +-- peers, so running SUM jumps by the peer-group total at each peer +-- rather than row-by-row. +-- ============================================================ + +query +SELECT player, game, score, + SUM(score) OVER (PARTITION BY player ORDER BY score) AS run_sum, + COUNT(score) OVER (PARTITION BY player ORDER BY score) AS run_count +FROM scores + +-- ============================================================ +-- 5.5: ROWS frame vs default RANGE frame over peer groups +-- ROWS frame is position-based: running SUM moves row-by-row ignoring peers. +-- The ROWS ordering adds `game` as a tie-breaker to make per-row output +-- deterministic (ROWS with tied ORDER BY can assign partial sums to peers +-- in either order). RANGE intentionally keeps ORDER BY score only, since +-- peer rows share the frame and therefore produce identical partial sums +-- regardless of intra-peer order. +-- ============================================================ + +query +SELECT player, game, score, + SUM(score) OVER (PARTITION BY player ORDER BY score, game + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS rows_sum, + SUM(score) OVER (PARTITION BY player ORDER BY score + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS range_sum +FROM scores + +-- ############################################################ +-- Section 6: Other window / aggregate functions +-- ############################################################ +-- Functions beyond Comet's native window-aggregate support +-- (Count/Min/Max/Sum/Average/First/Last). Everything below either runs natively +-- with a known correctness issue (NTILE) or falls back because Comet's native +-- WindowExec serde does not recognise the aggregate. + +-- ============================================================ +-- 6.1: NTILE (known correctness bug tracked in #4255) +-- Falls back to Spark via the NTile guard in CometWindowExec. +-- When the native bug is fixed and the guard removed, this test will fail +-- because Comet stops falling back — that's the signal to re-enable it. +-- ============================================================ + +query expect_fallback(NTILE has a correctness bug in Comet tracked in #4255) +SELECT dept, id, salary, + NTILE(4) OVER (PARTITION BY dept ORDER BY id) AS bucket +FROM emp + +-- ============================================================ +-- 6.2: statistical aggregates over a window (STDDEV / VAR / SKEW / KURT) +-- ============================================================ + +query expect_fallback(is not supported for window function) +SELECT dept, id, salary, + STDDEV_POP(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS sd_pop, + STDDEV_SAMP(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS sd_samp, + VAR_POP(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS v_pop, + VAR_SAMP(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS v_samp, + SKEWNESS(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS skew, + KURTOSIS(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS kurt +FROM emp + +-- ============================================================ +-- 6.3: collection aggregates (COLLECT_LIST) +-- id is unique per partition, so the ORDER BY makes COLLECT_LIST insertion +-- order deterministic. +-- ============================================================ + +query expect_fallback(is not supported for window function) +SELECT dept, id, salary, + COLLECT_LIST(salary) OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS so_far +FROM emp + +-- ============================================================ +-- 6.4: bitwise aggregates (BIT_AND / BIT_OR / BIT_XOR) +-- ============================================================ + +query expect_fallback(is not supported for window function) +SELECT dept, id, salary, + BIT_AND(salary) OVER (PARTITION BY dept) AS b_and, + BIT_OR(salary) OVER (PARTITION BY dept) AS b_or, + BIT_XOR(salary) OVER (PARTITION BY dept) AS b_xor +FROM emp + +-- ============================================================ +-- 6.5: correlation / covariance (CORR / COVAR_POP / COVAR_SAMP) +-- ============================================================ + +query expect_fallback(is not supported for window function) +SELECT dept, id, salary, hire_yr, + CORR(salary, hire_yr) OVER (PARTITION BY dept) AS corr, + COVAR_POP(salary, hire_yr) OVER (PARTITION BY dept) AS covp, + COVAR_SAMP(salary, hire_yr) OVER (PARTITION BY dept) AS covs +FROM emp + +-- ============================================================ +-- 6.6: percentile aggregates (PERCENTILE / MEDIAN / APPROX_PERCENTILE) +-- ============================================================ + +query expect_fallback(is not supported for window function) +SELECT dept, id, salary, + PERCENTILE(salary, 0.5) OVER (PARTITION BY dept) AS p50, + MEDIAN(salary) OVER (PARTITION BY dept) AS med, + APPROX_PERCENTILE(salary, 0.5) OVER (PARTITION BY dept) AS app_p50 +FROM emp + +-- ============================================================ +-- 6.7: value-selection aggregates (ANY_VALUE / MAX_BY / MIN_BY / MODE) +-- ============================================================ + +query expect_fallback(is not supported for window function) +SELECT dept, id, salary, + ANY_VALUE(salary) OVER (PARTITION BY dept ORDER BY id, salary) AS anyv, + MAX_BY(id, salary) OVER (PARTITION BY dept) AS max_id_by_salary, + MIN_BY(id, salary) OVER (PARTITION BY dept) AS min_id_by_salary, + MODE(salary) OVER (PARTITION BY dept) AS mode_salary +FROM emp + +-- ############################################################ +-- Section 7: LAG / LEAD respect null values (mirrors Spark +-- SQLWindowFunctionSuite "lead/lag should respect null values") +-- ############################################################ + +statement +CREATE TABLE lag_lead_nulls(a int, b int, c int) USING parquet + +statement +INSERT INTO lag_lead_nulls VALUES + (CAST(NULL AS INT), 1, 3), + (CAST(NULL AS INT), 2, 4) + +-- ============================================================ +-- 7.1: LAG / LEAD with literal default — input is null but the +-- default still fires when the offset row does not exist +-- (RESPECT NULLS is the default behaviour). +-- ============================================================ + +query +SELECT + b, + LAG(a, 1, 321) OVER (ORDER BY b) AS lg, + LEAD(a, 1, 321) OVER (ORDER BY b) AS ld +FROM lag_lead_nulls + +-- ============================================================ +-- 7.2: LAG / LEAD with a non-literal (column-reference) default +-- falls back to Spark — Comet's native lag/lead only accepts a +-- literal default value. +-- ============================================================ + +query expect_fallback(default value must be a literal) +SELECT + b, + LAG(a, 1, c) OVER (ORDER BY b) AS lg, + LEAD(a, 1, c) OVER (ORDER BY b) AS ld +FROM lag_lead_nulls diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index e5ddbc5b9a..c2c469e96d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate @@ -56,4 +56,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 53 eligible operators (69%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index 1ce50f8385..caecf50309 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate @@ -57,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 53 eligible operators (62%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_datafusion/extended.txt index 138d841ed0..0cec7e115e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_datafusion/extended.txt @@ -1,71 +1,69 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -88,4 +86,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 86 eligible operators (74%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 86 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt index 2044df0e8a..d343ec0ae7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt @@ -1,72 +1,70 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -89,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 63 out of 86 eligible operators (73%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt index a6cc526735..6b79b21f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -27,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index eb85a80eb1..e56490cb97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index 24498162ca..cc42c45f5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -27,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index 643e47b0c6..c7161dd543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt index 9480047251..91b1c49a79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 34 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index e434c5ff76..a77cdebfab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt index d2f31a9b37..b9a5a6e3eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt @@ -1,63 +1,60 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometProject + : : : +- CometFilter + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometSort + : : +- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 53 out of 55 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt index 259056f9f6..5b4da5622b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt @@ -1,63 +1,60 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometProject + : : : +- CometFilter + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometSort + : : +- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 53 out of 55 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index 54fa1daf2e..20697609d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt @@ -1,76 +1,74 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -99,4 +97,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 97 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index 9691120d2f..b605a0f915 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index cd6c39e4f3..26438c9ca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -1,93 +1,90 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 87 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index 0b4672eabb..615f10ad6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -1,94 +1,91 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index f67067735c..9f54e03733 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -1,52 +1,50 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 47 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index ea746c5fee..146bfb2df0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -1,53 +1,51 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 9a8a9bbba4..de381dfff4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index 093505a6cc..1a4a5c1e9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index aa4ccf3246..c46aca28f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt @@ -1,76 +1,74 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -99,4 +97,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 97 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index 5e8acff889..780362659c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt index 9a8a9bbba4..de381dfff4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index 093505a6cc..1a4a5c1e9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index 00dc622681..16126921c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 34 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index da0bef861d..47a9fb1e20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index c1d5342c8b..a4eda1ee01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -1,58 +1,55 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 730d37ec0a..fe1f4b42f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -1,59 +1,56 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 52 eligible operators (63%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index bddf80b4d0..e10c157987 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index c0056e2382..d64008b0a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -29,4 +29,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt index 9a8a9bbba4..de381dfff4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index 093505a6cc..1a4a5c1e9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index f1fc5ff9a3..2cd06214ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt @@ -1,33 +1,32 @@ CometNativeColumnarToRow +- CometProject +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index a18137ef10..ff80b7a2aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -1,34 +1,33 @@ CometNativeColumnarToRow +- CometProject +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index 44999fda71..0255bcdcba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -165,4 +165,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 100 out of 156 eligible operators (64%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 103 out of 156 eligible operators (66%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index ff6d42786b..d43b31589d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -168,4 +168,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 97 out of 156 eligible operators (62%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 156 eligible operators (64%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_datafusion/extended.txt index 138d841ed0..0cec7e115e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_datafusion/extended.txt @@ -1,71 +1,69 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -88,4 +86,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 86 eligible operators (74%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 86 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt index 2044df0e8a..d343ec0ae7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt @@ -1,72 +1,70 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -89,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 63 out of 86 eligible operators (73%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index a6cc526735..6b79b21f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -27,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index eb85a80eb1..e56490cb97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index 24498162ca..cc42c45f5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -27,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index 643e47b0c6..c7161dd543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index be0d09e04e..1427cfbe10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -99,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 96 out of 99 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index e9571a1f12..672bf0c9e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt index 54fa1daf2e..20697609d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt @@ -1,76 +1,74 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -99,4 +97,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 97 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index 9691120d2f..b605a0f915 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index cd6c39e4f3..26438c9ca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -1,93 +1,90 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 87 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index 0b4672eabb..615f10ad6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -1,94 +1,91 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 36a797bad9..360045edb6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -1,208 +1,199 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Exchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 188 out of 196 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index 190830204f..4114ab6309 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -1,212 +1,203 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Exchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 134 out of 196 eligible operators (68%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 184 out of 196 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt index aa4ccf3246..c46aca28f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt @@ -1,76 +1,74 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -99,4 +97,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 97 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index 5e8acff889..780362659c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index 19b0999bb5..04ccad38d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometUnion @@ -282,4 +282,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 270 out of 282 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 273 out of 282 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index b610537f40..1c7d9b3c05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometUnion @@ -291,4 +291,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 264 out of 282 eligible operators (93%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index dab9824951..4e699a0b40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -1,165 +1,156 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 100 out of 153 eligible operators (65%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 147 out of 153 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index a9c5d2e774..7a69abd7f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -1,168 +1,159 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 97 out of 153 eligible operators (63%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 144 out of 153 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt index 4c66209a5e..a9e3b1bd8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -81,4 +81,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 81 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index 58d00a427d..cd137e3d1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -84,4 +84,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt index ccf9f25336..d7e9b9bc5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt @@ -1,32 +1,31 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index 1a985ead1b..cbc9796ed9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -1,33 +1,32 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala index 23acc2b16d..777211e858 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.Tag import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, Row} import org.apache.spark.sql.comet.CometWindowExec -import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions.{count, lead, sum} @@ -43,6 +42,10 @@ class CometWindowExecSuite extends CometTestBase { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_WINDOW_ENABLED.key -> "true", + "spark.comet.operator.WindowExec.allowIncompatible" -> "true", + "spark.comet.explainFallback.enabled" -> "true", + "spark.comet.logFallbackReasons.enabled" -> "true", + "spark.comet.exec.localTableScan.enabled" -> "true", CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { testFun } @@ -54,14 +57,14 @@ class CometWindowExecSuite extends CometTestBase { CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { - checkSparkAnswer(sql(""" + checkSparkAnswerAndOperator(sql(""" |SELECT | lag(123, 100, 321) OVER (ORDER BY id) as lag, | lead(123, 100, 321) OVER (ORDER BY id) as lead |FROM (SELECT 1 as id) tmp """.stripMargin)) - checkSparkAnswer(sql(""" + checkSparkAnswerAndOperator(sql(""" |SELECT | lag(123, 100, a) OVER (ORDER BY id) as lag, | lead(123, 100, a) OVER (ORDER BY id) as lead @@ -76,18 +79,32 @@ class CometWindowExecSuite extends CometTestBase { val df = Seq(1, 2, 4, 3, 2, 1).toDF("value") val window = Window.orderBy($"value".desc) - // ranges are long - val df2 = df.select( - $"value", - sum($"value").over(window.rangeBetween(Window.unboundedPreceding, 1L)), - sum($"value").over(window.rangeBetween(1L, Window.unboundedFollowing))) + // ranges are long. Spark encodes PRECEDING/FOLLOWING via the sign of the bound; + // `rangeBetween(unboundedPreceding, 1L)` produces upper=1 FOLLOWING, which is + // representable in our proto and runs natively. + val df2 = + df.select($"value", sum($"value").over(window.rangeBetween(Window.unboundedPreceding, 1L))) - // Comet does not support RANGE BETWEEN - // https://github.com/apache/datafusion-comet/issues/1246 - val (_, cometPlan) = checkSparkAnswer(df2) + val (_, cometPlan) = checkSparkAnswerAndOperator(df2) val cometWindowExecs = collect(cometPlan) { case w: CometWindowExec => w } + assert(cometWindowExecs.nonEmpty) + } + + test("window query with rangeBetween FOLLOWING lower bound falls back to Spark") { + // `rangeBetween(1L, unboundedFollowing)` puts a positive offset (FOLLOWING semantic) + // in the lower bound position, which the proto only encodes as Preceding. We fall + // back to Spark rather than misinterpret the bound. + val df = Seq(1, 2, 4, 3, 2, 1).toDF("value") + val window = Window.orderBy($"value".desc) + val df2 = + df.select($"value", sum($"value").over(window.rangeBetween(1L, Window.unboundedFollowing))) + + checkSparkAnswer(df2) + val cometWindowExecs = collect(df2.queryExecution.executedPlan) { case w: CometWindowExec => + w + } assert(cometWindowExecs.isEmpty) } @@ -105,19 +122,7 @@ class CometWindowExecSuite extends CometTestBase { |select month, area, product, sum(product + 1) over (partition by 1 order by 2) |from windowData """.stripMargin) - checkSparkAnswer(df2) - val cometShuffles = collect(df2.queryExecution.executedPlan) { - case _: CometShuffleExchangeExec => true - } - if (shuffleMode == "jvm" || shuffleMode == "auto") { - assert(cometShuffles.length == 1) - } else { - // we fall back to Spark for shuffle because we do not support - // native shuffle with a LocalTableScan input, and we do not fall - // back to Comet columnar shuffle due to - // https://github.com/apache/datafusion-comet/issues/1248 - assert(cometShuffles.isEmpty) - } + checkSparkAnswerAndOperator(df2) } } } @@ -134,7 +139,7 @@ class CometWindowExecSuite extends CometTestBase { val df = sql(""" SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg |""".stripMargin) - checkSparkAnswer(df) + checkSparkAnswerAndOperator(df) } } @@ -157,7 +162,7 @@ class CometWindowExecSuite extends CometTestBase { |SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) |FROM testData ORDER BY cate, val |""".stripMargin) - checkSparkAnswer(df1) + checkSparkAnswerAndOperator(df1) } } @@ -166,12 +171,12 @@ class CometWindowExecSuite extends CometTestBase { Seq((1L, "1"), (1L, "1"), (2147483650L, "1"), (3L, "2"), (2L, "1"), (2147483650L, "2")) .toDF("key", "value") - checkSparkAnswer( + checkSparkAnswerAndOperator( df.select( $"key", count("key").over( Window.partitionBy($"value").orderBy($"key").rangeBetween(0, 2147483648L)))) - checkSparkAnswer( + checkSparkAnswerAndOperator( df.select( $"key", count("key").over( @@ -192,7 +197,7 @@ class CometWindowExecSuite extends CometTestBase { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled, SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_DISTRIBUTION.key -> "true", - CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { val df = Seq(("a", 1, 1), ("a", 2, 2), ("b", 1, 3), ("b", 1, 4)).toDF("key1", "key2", "value") val windowSpec = Window.partitionBy("key1", "key2").orderBy("value") @@ -202,12 +207,12 @@ class CometWindowExecSuite extends CometTestBase { .repartition($"key1") .select(lead($"key1", 1).over(windowSpec), lead($"value", 1).over(windowSpec)) - checkSparkAnswer(windowed) + checkSparkAnswerAndOperator(windowed) } } } - ignore("aggregate window function for all types") { + test("aggregate window function for all types") { val numValues = 2048 Seq(1, 100, numValues).foreach { numGroups => @@ -219,20 +224,29 @@ class CometWindowExecSuite extends CometTestBase { Seq(128, numValues + 100).foreach { batchSize => withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { (1 to 11).foreach { col => + // _10 and _11 are TIMESTAMP columns; Spark allows SUM(timestamp) + // via an implicit cast to DOUBLE, which is semantically meaningless + // for a real query and introduces a Cast(TimestampType, DoubleType) + // that Comet does not support. Exclude SUM for those columns the + // same way _12 (DATE) is excluded below. val aggregateFunctions = - List(s"COUNT(_$col)", s"MAX(_$col)", s"MIN(_$col)", s"SUM(_$col)") + if (col == 10 || col == 11) { + List(s"COUNT(_$col)", s"MAX(_$col)", s"MIN(_$col)") + } else { + List(s"COUNT(_$col)", s"MAX(_$col)", s"MIN(_$col)", s"SUM(_$col)") + } aggregateFunctions.foreach { function => val df1 = sql(s"SELECT $function OVER() FROM tbl") - checkSparkAnswerWithTolerance(df1, 1e-6) + checkSparkAnswerAndOperatorWithTol(df1) val df2 = sql(s"SELECT $function OVER(order by _2) FROM tbl") - checkSparkAnswerWithTolerance(df2, 1e-6) + checkSparkAnswerAndOperatorWithTol(df2) val df3 = sql(s"SELECT $function OVER(order by _2 desc) FROM tbl") - checkSparkAnswerWithTolerance(df3, 1e-6) + checkSparkAnswerAndOperatorWithTol(df3) val df4 = sql(s"SELECT $function OVER(partition by _2 order by _2) FROM tbl") - checkSparkAnswerWithTolerance(df4, 1e-6) + checkSparkAnswerAndOperatorWithTol(df4) } } @@ -240,16 +254,16 @@ class CometWindowExecSuite extends CometTestBase { val aggregateFunctionsWithoutSum = List("COUNT(_12)", "MAX(_12)", "MIN(_12)") aggregateFunctionsWithoutSum.foreach { function => val df1 = sql(s"SELECT $function OVER() FROM tbl") - checkSparkAnswerWithTolerance(df1, 1e-6) + checkSparkAnswerAndOperatorWithTol(df1) val df2 = sql(s"SELECT $function OVER(order by _2) FROM tbl") - checkSparkAnswerWithTolerance(df2, 1e-6) + checkSparkAnswerAndOperatorWithTol(df2) val df3 = sql(s"SELECT $function OVER(order by _2 desc) FROM tbl") - checkSparkAnswerWithTolerance(df3, 1e-6) + checkSparkAnswerAndOperatorWithTol(df3) val df4 = sql(s"SELECT $function OVER(partition by _2 order by _2) FROM tbl") - checkSparkAnswerWithTolerance(df4, 1e-6) + checkSparkAnswerAndOperatorWithTol(df4) } } } @@ -259,7 +273,7 @@ class CometWindowExecSuite extends CometTestBase { } } - ignore("Windows support") { + test("Windows support") { Seq("true", "false").foreach(aqeEnabled => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", @@ -285,9 +299,7 @@ class CometWindowExecSuite extends CometTestBase { s"SELECT $function OVER(order by _2 rows between current row and 1 following) FROM t1") queries.foreach { query => - checkSparkAnswerAndFallbackReason( - query, - "Native WindowExec has known correctness issues") + checkSparkAnswerAndOperator(query) } } } @@ -306,7 +318,7 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql("SELECT a, b, c, COUNT(*) OVER () as cnt FROM window_test") - checkSparkAnswerAndFallbackReason(df, "Native WindowExec has known correctness issues") + checkSparkAnswerAndOperator(df) } } @@ -322,13 +334,11 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql("SELECT a, b, c, SUM(c) OVER (PARTITION BY a) as sum_c FROM window_test") - checkSparkAnswerAndFallbackReason(df, "Native WindowExec has known correctness issues") + checkSparkAnswerAndOperator(df) } } - // TODO: AVG with PARTITION BY and ORDER BY not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: AVG with PARTITION BY and ORDER BY") { + test("window: AVG with PARTITION BY and ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -362,13 +372,11 @@ class CometWindowExecSuite extends CometTestBase { MAX(c) OVER (ORDER BY b) as max_c FROM window_test """) - checkSparkAnswerAndFallbackReason(df, "Native WindowExec has known correctness issues") + checkSparkAnswerAndOperator(df) } } - // TODO: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW produces incorrect results - // Returns wrong cnt values - ordering issue causes swapped values for rows with same partition - ignore("window: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { + test("window: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -379,22 +387,33 @@ class CometWindowExecSuite extends CometTestBase { .parquet(dir.toString) spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") + // Excluding c column from the result dataset + // as there is no output order guarantee for both Spark or DataFusion + // because C column is not part of partitioning and sorting val df = sql(""" - SELECT a, b, c, - COUNT(*) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) as cnt - FROM window_test + select a, b, cnt from( + SELECT a, b, c, + COUNT(*) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) as cnt + FROM window_test + ) """) checkSparkAnswerAndOperator(df) + + val df1 = sql(""" + SELECT a, b, c, + COUNT(*) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) as cnt + FROM window_test + """) + checkSparkAnswerAndOperator(df1) } } - // TODO: SUM with ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING produces incorrect results - ignore("window: SUM with ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING") { + test("window: SUM with ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) .toDF("a", "b", "c") - .repartition(3) + .repartition(1) .write .mode("overwrite") .parquet(dir.toString) @@ -402,16 +421,14 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" SELECT a, b, c, - SUM(c) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) as sum_c + SUM(c) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) as sum_c FROM window_test """) checkSparkAnswerAndOperator(df) } } - // TODO: AVG with ROWS BETWEEN produces incorrect results - // Returns wrong avg_c values - calculation appears to be off - ignore("window: AVG with ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING") { + test("window: AVG with ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -424,15 +441,14 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" SELECT a, b, c, - AVG(c) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as avg_c + AVG(c) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as avg_c FROM window_test """) checkSparkAnswerAndOperator(df) } } - // TODO: SUM with ROWS BETWEEN produces incorrect results - ignore("window: SUM with ROWS BETWEEN 2 PRECEDING AND CURRENT ROW") { + test("window: SUM with ROWS BETWEEN 2 PRECEDING AND CURRENT ROW") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -445,16 +461,14 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" SELECT a, b, c, - SUM(c) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) as sum_c + SUM(c) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) as sum_c FROM window_test """) checkSparkAnswerAndOperator(df) } } - // TODO: COUNT with ROWS BETWEEN not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: COUNT with ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING") { + test("window: COUNT with ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -467,16 +481,14 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" SELECT a, b, c, - COUNT(*) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING) as cnt + COUNT(*) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING) as cnt FROM window_test """) checkSparkAnswerAndOperator(df) } } - // TODO: MAX with ROWS BETWEEN UNBOUNDED not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: MAX with ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING") { + test("window: MAX with ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -496,9 +508,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: ROW_NUMBER not supported - // Falls back to Spark Window operator - ignore("window: ROW_NUMBER with PARTITION BY and ORDER BY") { + test("window: ROW_NUMBER with PARTITION BY and ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -518,9 +528,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: RANK not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: RANK with PARTITION BY and ORDER BY") { + test("window: RANK with PARTITION BY and ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -540,9 +548,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: DENSE_RANK not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: DENSE_RANK with PARTITION BY and ORDER BY") { + test("window: DENSE_RANK with PARTITION BY and ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -562,9 +568,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: PERCENT_RANK not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: PERCENT_RANK with PARTITION BY and ORDER BY") { + test("window: PERCENT_RANK with PARTITION BY and ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -584,9 +588,11 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: NTILE not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: NTILE with PARTITION BY and ORDER BY") { + test("window: NTILE with PARTITION BY and ORDER BY") { + // Correctness issue bug tracked in https://github.com/apache/datafusion-comet/issues/4255. + // CometWindowExec forces NTILE to fall back to Spark via a guard; this test + // pins that behavior so it will fail (alerting us to re-enable native NTILE) + // once the underlying correctness issue is fixed and the guard removed. withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -598,11 +604,13 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" - SELECT a, b, c, + SELECT a, b, NTILE(4) OVER (PARTITION BY a ORDER BY b) as ntile_4 FROM window_test """) - checkSparkAnswerAndOperator(df) + checkSparkAnswerAndFallbackReason( + df, + "NTILE has a correctness bug in Comet tracked in #4255") } } @@ -719,13 +727,14 @@ class CometWindowExecSuite extends CometTestBase { withTempDir { dir => Seq((1, 1, Some(10)), (1, 2, None), (1, 3, Some(30)), (2, 1, None), (2, 2, Some(20))) .toDF("a", "b", "c") + .repartition(3) .write .mode("overwrite") .parquet(dir.toString) spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" - SELECT a, b, c, + SELECT a, b, LEAD(c) IGNORE NULLS OVER (PARTITION BY a ORDER BY b) as lead_c FROM window_test """) @@ -734,12 +743,11 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: FIRST_VALUE causes encoder error - // org.apache.spark.SparkUnsupportedOperationException: [ENCODER_NOT_FOUND] Not found an encoder of the type Any - ignore("window: FIRST_VALUE with default ignore nulls") { + test("window: FIRST_VALUE with default ignore nulls") { withTempDir { dir => (0 until 30) - .map(i => (i % 3, i % 5, if (i % 7 == 0) null else i)) + .map(i => + (i % 3, i % 5, if (i % 7 == 0) null.asInstanceOf[Integer] else Integer.valueOf(i))) .toDF("a", "b", "c") .repartition(3) .write @@ -756,12 +764,11 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: LAST_VALUE causes encoder error - // org.apache.spark.SparkUnsupportedOperationException: [ENCODER_NOT_FOUND] Not found an encoder of the type Any - ignore("window: LAST_VALUE with ROWS frame") { + test("window: LAST_VALUE with ROWS frame") { withTempDir { dir => (0 until 30) - .map(i => (i % 3, i % 5, if (i % 7 == 0) null else i)) + .map(i => + (i % 3, i % 5, if (i % 7 == 0) null.asInstanceOf[Integer] else Integer.valueOf(i))) .toDF("a", "b", "c") .repartition(3) .write @@ -778,8 +785,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: NTH_VALUE returns incorrect results - produces 0 instead of null for first row, - ignore("window: NTH_VALUE with position 2") { + test("window: NTH_VALUE with position 2") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -791,17 +797,15 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" - SELECT a, b, c, - NTH_VALUE(c, 2) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) as nth_c + SELECT a, b, + NTH_VALUE(c, 2) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) as nth_c FROM window_test """) checkSparkAnswerAndOperator(df) } } - // TODO: CUME_DIST not supported - falls back to Spark Window operator - // Error: "Partitioning and sorting specifications must be the same" - ignore("window: CUME_DIST with PARTITION BY and ORDER BY") { + test("window: CUME_DIST with PARTITION BY and ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -821,8 +825,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Multiple window functions with mixed frame types (RowFrame and RangeFrame) - ignore("window: multiple window functions in single query") { + test("window: multiple window functions in single query") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -834,7 +837,7 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" - SELECT a, b, c, + SELECT a, b, ROW_NUMBER() OVER (PARTITION BY a ORDER BY b) as row_num, RANK() OVER (PARTITION BY a ORDER BY b) as rnk, SUM(c) OVER (PARTITION BY a ORDER BY b) as sum_c, @@ -845,9 +848,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Different window specifications not fully supported - // Falls back to Spark Project and Window operators - ignore("window: different window specifications in single query") { + test("window: different window specifications in single query") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -869,9 +870,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: ORDER BY DESC with aggregation not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: ORDER BY DESC with aggregation") { + test("window: ORDER BY DESC with aggregation") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -891,9 +890,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Multiple PARTITION BY columns not supported - // Falls back to Spark Window operator - ignore("window: multiple PARTITION BY columns") { + test("window: multiple PARTITION BY columns") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i % 2, i)) @@ -913,9 +910,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Multiple ORDER BY columns not supported - // Falls back to Spark Window operator - ignore("window: multiple ORDER BY columns") { + test("window: multiple ORDER BY columns") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i % 2, i)) @@ -935,9 +930,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: RANGE BETWEEN with numeric ORDER BY not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: RANGE BETWEEN with numeric ORDER BY") { + test("window: RANGE BETWEEN with numeric ORDER BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i, i * 2)) @@ -957,9 +950,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { + test("window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i, i * 2)) @@ -979,9 +970,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Complex expressions in window functions not fully supported - // Falls back to Spark Project operator - ignore("window: complex expression in window function") { + test("window: complex expression in window function") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -1001,9 +990,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Window function with WHERE clause not supported - // Falls back to Spark Window operator - "Partitioning and sorting specifications must be the same" - ignore("window: window function with WHERE clause") { + test("window: window function with WHERE clause") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -1024,9 +1011,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: Window function with GROUP BY not fully supported - // Falls back to Spark Project and Window operators - ignore("window: window function with GROUP BY") { + test("window: window function with GROUP BY") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -1047,8 +1032,7 @@ class CometWindowExecSuite extends CometTestBase { } } - // TODO: ROWS BETWEEN with negative offset produces incorrect results - ignore("window: ROWS BETWEEN with negative offset") { + test("window: ROWS BETWEEN with negative offset") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -1060,16 +1044,15 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" - SELECT a, b, c, - AVG(c) OVER (PARTITION BY a ORDER BY b ROWS BETWEEN 3 PRECEDING AND 1 PRECEDING) as avg_c + SELECT a, b, + AVG(c) OVER (PARTITION BY a ORDER BY b, c ROWS BETWEEN 3 PRECEDING AND 1 PRECEDING) as avg_c FROM window_test """) checkSparkAnswerAndOperator(df) } } - // TODO: All ranking functions together produce incorrect row_num values - ignore("window: all ranking functions together") { + test("window: all ranking functions together") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -1081,7 +1064,7 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql(""" - SELECT a, b, c, + SELECT a, b, ROW_NUMBER() OVER (PARTITION BY a ORDER BY b) as row_num, RANK() OVER (PARTITION BY a ORDER BY b) as rnk, DENSE_RANK() OVER (PARTITION BY a ORDER BY b) as dense_rnk, @@ -1090,6 +1073,46 @@ class CometWindowExecSuite extends CometTestBase { NTILE(3) OVER (PARTITION BY a ORDER BY b) as ntile_3 FROM window_test """) + // all functions ok excepting NTILE + checkSparkAnswerAndFallbackReason( + df, + "NTILE has a correctness bug in Comet tracked in #4255") + } + } + + test("window: FIRST_VALUE/LAST_VALUE/NTH_VALUE IGNORE NULLS with leading nulls") { + withTempDir { dir => + Seq( + ("eng", 1, Option(100), 2020), + ("eng", 2, Option(100), 2021), + ("eng", 3, Option(150), 2022), + ("eng", 4, Option.empty[Int], 2023), + ("eng", 5, Option(200), 2024), + ("sales", 6, Option(90), 2020), + ("sales", 7, Option(90), 2021), + ("sales", 8, Option.empty[Int], 2022), + ("sales", 9, Option(110), 2023), + ("sales", 10, Option(120), 2024), + ("ops", 11, Option.empty[Int], 2020), + ("ops", 12, Option.empty[Int], 2021), + ("ops", 13, Option(50), 2022)) + .toDF("dept", "id", "salary", "hire_yr") + .write + .mode("overwrite") + .parquet(dir.toString) + + spark.read.parquet(dir.toString).createOrReplaceTempView("emp") + val df = sql(""" + SELECT dept, id, salary, + /*first_value(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS fv_ignore, + nth_value(salary, 1) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS nv_ignore, */ + last_value(salary) IGNORE NULLS OVER (PARTITION BY dept ORDER BY id, salary + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS lv_ignore + + FROM emp + """) checkSparkAnswerAndOperator(df) } }