Skip to content
  •  
  •  
  •  
86 changes: 74 additions & 12 deletions spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,13 @@ object CometExecRule {

val allExecs: Map[Class[_ <: SparkPlan], CometOperatorSerde[_]] = nativeExecs ++ sinks

/**
* Tag set on a `ShuffleExchangeExec` that should be left as a plain Spark shuffle rather than
* wrapped in `CometShuffleExchangeExec`. See `tagRedundantColumnarShuffle`.
*/
val SKIP_COMET_SHUFFLE_TAG: org.apache.spark.sql.catalyst.trees.TreeNodeTag[Unit] =
org.apache.spark.sql.catalyst.trees.TreeNodeTag[Unit]("comet.skipCometShuffle")

}

/**
Expand All @@ -97,19 +104,66 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] {

private lazy val showTransformations = CometConf.COMET_EXPLAIN_TRANSFORMATIONS.get()

/**
* Revert any `CometShuffleExchangeExec` with `CometColumnarShuffle` that is sandwiched between
* two non-Comet `HashAggregateExec` / `ObjectHashAggregateExec` operators back to the original
* Spark `ShuffleExchangeExec`. This is the partial-final-aggregate pattern where Comet couldn't
* convert either aggregate; keeping a columnar shuffle between them only adds
* row->arrow->shuffle->arrow->row conversion overhead with no Comet consumer on either side.
* See https://github.com/apache/datafusion-comet/issues/4004.
*
* The match is intentionally narrow (both sides must be row-based aggregates that remained JVM
* after the main transform pass). Running the revert post-transform means we only fire when the
* main conversion already decided to keep both aggregates JVM - we never create the dangerous
* mixed mode where a Comet partial feeds a JVM final (see issue #1389).
*
* Also tag the reverted shuffle so AQE stage-isolated re-planning does not convert it back to a
* Comet shuffle when the outer aggregate context is no longer visible.
*/
private def revertRedundantColumnarShuffle(plan: SparkPlan): SparkPlan = {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the opttimzation @andygrove
I guess this is the first time where CometShuffleExchangeExec is reverted back to a plain ShuffleExchangeExec.

The two shuffle paths use different memory systems:

  • Comet columnar shuffle uses Comet's own memory pool. (off-heap)
  • Spark vanilla shuffle uses the JVM execution memory pool , with spills managed by ExternalSorter.

Users who have tuned their clusters for Comet (smaller JVM heap) could see unexpected spills after this chang, shifting shuffle memory pressure back to theJVM.
Additionally, Comet's Arrow IPC columnar format typically compresses better than Spark's row-based UnsafeRowSerializer path, so shuffle I/O mayalso increase.
It would be good to document or log when a shuffle is reverted so users can correlate any unexpected behavior with this optimization.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the feedback @karuppayya. I'm assuming the cost of doing two transitions (r2c then c2r) would outweigh the benefits of using Comet shuffle? I agree that it would be worth adding documentation.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree that this optimization will improve performance and compute efficiency.

My main concern is determining the best recommendation for users to tune memory, particularly since they cannot explicitly disable it.

Also can it be a seperate rule in itself and have it only in org.apache.comet.CometSparkSessionExtensions.CometExecColumnar#postColumnarTransitions?

def isAggregate(p: SparkPlan): Boolean =
p.isInstanceOf[HashAggregateExec] || p.isInstanceOf[ObjectHashAggregateExec]

def isRedundantShuffle(child: SparkPlan): Boolean = child match {
case s: CometShuffleExchangeExec =>
s.shuffleType == CometColumnarShuffle && isAggregate(s.child)
case _ => false
}

plan.transform {
case op if isAggregate(op) && op.children.exists(isRedundantShuffle) =>
val newChildren = op.children.map {
case s: CometShuffleExchangeExec
if s.shuffleType == CometColumnarShuffle && isAggregate(s.child) =>
val reverted =
s.originalPlan.withNewChildren(Seq(s.child)).asInstanceOf[ShuffleExchangeExec]
reverted.setTagValue(CometExecRule.SKIP_COMET_SHUFFLE_TAG, ())
reverted
case other => other
}
op.withNewChildren(newChildren)
}
}

private def shouldSkipCometShuffle(s: ShuffleExchangeExec): Boolean =
s.getTagValue(CometExecRule.SKIP_COMET_SHUFFLE_TAG).isDefined

private def applyCometShuffle(plan: SparkPlan): SparkPlan = {
plan.transformUp { case s: ShuffleExchangeExec =>
CometShuffleExchangeExec.shuffleSupported(s) match {
case Some(CometNativeShuffle) =>
// Switch to use Decimal128 regardless of precision, since Arrow native execution
// doesn't support Decimal32 and Decimal64 yet.
conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true")
CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle)
case Some(CometColumnarShuffle) =>
CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle)
case None =>
s
}
plan.transformUp {
case s: ShuffleExchangeExec if shouldSkipCometShuffle(s) =>
s
case s: ShuffleExchangeExec =>
CometShuffleExchangeExec.shuffleSupported(s) match {
case Some(CometNativeShuffle) =>
// Switch to use Decimal128 regardless of precision, since Arrow native execution
// doesn't support Decimal32 and Decimal64 yet.
conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true")
CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle)
case Some(CometColumnarShuffle) =>
CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle)
case None =>
s
}
}
}

Expand Down Expand Up @@ -258,6 +312,9 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] {
case s @ ShuffleQueryStageExec(_, ReusedExchangeExec(_, _: CometShuffleExchangeExec), _) =>
convertToComet(s, CometExchangeSink).getOrElse(s)

case s: ShuffleExchangeExec if shouldSkipCometShuffle(s) =>
s

case s: ShuffleExchangeExec =>
convertToComet(s, CometShuffleExchangeExec).getOrElse(s)

Expand Down Expand Up @@ -409,6 +466,11 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] {
case CometScanWrapper(_, s) => s
}

// Revert CometColumnarShuffle to Spark's ShuffleExchangeExec when sandwiched between two
// non-Comet HashAggregate/ObjectHashAggregate operators that remained JVM after the main
// transform pass. See https://github.com/apache/datafusion-comet/issues/4004.
newPlan = revertRedundantColumnarShuffle(newPlan)

// Set up logical links
newPlan = newPlan.transform {
case op: CometExec =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,46 +7,43 @@ TakeOrderedAndProject
: : +- BroadcastHashJoin
: : :- Filter
: : : +- HashAggregate
: : : +- CometNativeColumnarToRow
: : : +- CometColumnarExchange
: : : +- HashAggregate
: : : +- Project
: : : +- BroadcastHashJoin
: : : :- Filter
: : : : +- ColumnarToRow
: : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : : +- SubqueryBroadcast
: : : : +- BroadcastExchange
: : : : +- CometNativeColumnarToRow
: : : : +- CometProject
: : : : +- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- BroadcastExchange
: : : +- CometNativeColumnarToRow
: : : +- CometProject
: : : +- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- Exchange
: : : +- HashAggregate
: : : +- Project
: : : +- BroadcastHashJoin
: : : :- Filter
: : : : +- ColumnarToRow
: : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : : +- SubqueryBroadcast
: : : : +- BroadcastExchange
: : : : +- CometNativeColumnarToRow
: : : : +- CometProject
: : : : +- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- BroadcastExchange
: : : +- CometNativeColumnarToRow
: : : +- CometProject
: : : +- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : +- BroadcastExchange
: : +- Filter
: : +- HashAggregate
: : +- CometNativeColumnarToRow
: : +- CometColumnarExchange
: : +- Exchange
: : +- HashAggregate
: : +- HashAggregate
: : +- HashAggregate
: : +- CometNativeColumnarToRow
: : +- CometColumnarExchange
: : +- HashAggregate
: : +- Project
: : +- BroadcastHashJoin
: : :- Filter
: : : +- ColumnarToRow
: : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : +- ReusedSubquery
: : +- BroadcastExchange
: : +- CometNativeColumnarToRow
: : +- CometProject
: : +- CometFilter
: : +- CometNativeScan parquet spark_catalog.default.date_dim
: : +- Exchange
: : +- HashAggregate
: : +- Project
: : +- BroadcastHashJoin
: : :- Filter
: : : +- ColumnarToRow
: : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : +- ReusedSubquery
: : +- BroadcastExchange
: : +- CometNativeColumnarToRow
: : +- CometProject
: : +- CometFilter
: : +- CometNativeScan parquet spark_catalog.default.date_dim
: +- BroadcastExchange
: +- CometNativeColumnarToRow
: +- CometProject
Expand All @@ -58,4 +55,4 @@ TakeOrderedAndProject
+- CometFilter
+- CometNativeScan parquet spark_catalog.default.customer

Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet.
Comet accelerated 15 out of 49 eligible operators (30%). Final plan contains 7 transitions between Spark and Comet.
Original file line number Diff line number Diff line change
@@ -1,67 +1,66 @@
TakeOrderedAndProject
+- HashAggregate
+- CometNativeColumnarToRow
+- CometColumnarExchange
+- HashAggregate
+- Project
+- BroadcastHashJoin
:- Project
: +- BroadcastHashJoin
: :- Project
: : +- Filter
: : +- BroadcastHashJoin
: : :- BroadcastHashJoin
: : : :- BroadcastHashJoin
: : : : :- CometNativeColumnarToRow
: : : : : +- CometFilter
: : : : : +- CometNativeScan parquet spark_catalog.default.customer
: : : : +- BroadcastExchange
: : : : +- Project
: : : : +- BroadcastHashJoin
: : : : :- ColumnarToRow
: : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : : : +- SubqueryBroadcast
: : : : : +- BroadcastExchange
: : : : : +- CometNativeColumnarToRow
: : : : : +- CometProject
: : : : : +- CometFilter
: : : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : : +- BroadcastExchange
: : : : +- CometNativeColumnarToRow
: : : : +- CometProject
: : : : +- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- BroadcastExchange
: : : +- Project
: : : +- BroadcastHashJoin
: : : :- ColumnarToRow
: : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : : +- ReusedSubquery
: : : +- BroadcastExchange
: : : +- CometNativeColumnarToRow
: : : +- CometProject
: : : +- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : +- BroadcastExchange
: : +- Project
: : +- BroadcastHashJoin
: : :- ColumnarToRow
: : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : +- ReusedSubquery
: : +- BroadcastExchange
: : +- CometNativeColumnarToRow
: : +- CometProject
: : +- CometFilter
: : +- CometNativeScan parquet spark_catalog.default.date_dim
: +- BroadcastExchange
: +- CometNativeColumnarToRow
: +- CometProject
: +- CometFilter
: +- CometNativeScan parquet spark_catalog.default.customer_address
+- BroadcastExchange
+- CometNativeColumnarToRow
+- CometProject
+- CometFilter
+- CometNativeScan parquet spark_catalog.default.customer_demographics
+- Exchange
+- HashAggregate
+- Project
+- BroadcastHashJoin
:- Project
: +- BroadcastHashJoin
: :- Project
: : +- Filter
: : +- BroadcastHashJoin
: : :- BroadcastHashJoin
: : : :- BroadcastHashJoin
: : : : :- CometNativeColumnarToRow
: : : : : +- CometFilter
: : : : : +- CometNativeScan parquet spark_catalog.default.customer
: : : : +- BroadcastExchange
: : : : +- Project
: : : : +- BroadcastHashJoin
: : : : :- ColumnarToRow
: : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : : : +- SubqueryBroadcast
: : : : : +- BroadcastExchange
: : : : : +- CometNativeColumnarToRow
: : : : : +- CometProject
: : : : : +- CometFilter
: : : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : : +- BroadcastExchange
: : : : +- CometNativeColumnarToRow
: : : : +- CometProject
: : : : +- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- BroadcastExchange
: : : +- Project
: : : +- BroadcastHashJoin
: : : :- ColumnarToRow
: : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : : +- ReusedSubquery
: : : +- BroadcastExchange
: : : +- CometNativeColumnarToRow
: : : +- CometProject
: : : +- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : +- BroadcastExchange
: : +- Project
: : +- BroadcastHashJoin
: : :- ColumnarToRow
: : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning]
: : : +- ReusedSubquery
: : +- BroadcastExchange
: : +- CometNativeColumnarToRow
: : +- CometProject
: : +- CometFilter
: : +- CometNativeScan parquet spark_catalog.default.date_dim
: +- BroadcastExchange
: +- CometNativeColumnarToRow
: +- CometProject
: +- CometFilter
: +- CometNativeScan parquet spark_catalog.default.customer_address
+- BroadcastExchange
+- CometNativeColumnarToRow
+- CometProject
+- CometFilter
+- CometNativeScan parquet spark_catalog.default.customer_demographics

Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet.
Comet accelerated 20 out of 54 eligible operators (37%). Final plan contains 10 transitions between Spark and Comet.
Loading
Loading