Skip to content

Commit

Permalink
perf: Add experimental feature to replace SortMergeJoin with Shuffled…
Browse files Browse the repository at this point in the history
…HashJoin (#1007)

* experiment

* fix and add credit

* disable by default and make internal

* remove sort

* minor optimization

* minor optimization

* remove unused import

* disable feature by default

* fix dockerfile

* Add section to tuning guide

* update benchmarking guide

* Revert "chore: Reserve memory for native shuffle writer per partition (#988)"

This reverts commit e146cfa.

* mark feature as experimental and explain risks

* workaround for TPC-DS q14 hanging on a RightSemi join

* revert a change

* remove debug logging:

* format

* add link to tuning guide
  • Loading branch information
andygrove authored Oct 21, 2024
1 parent 3df9d5c commit cb3e977
Show file tree
Hide file tree
Showing 7 changed files with 110 additions and 2 deletions.
7 changes: 6 additions & 1 deletion NOTICE.txt
Original file line number Diff line number Diff line change
Expand Up @@ -5,4 +5,9 @@ This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).

This product includes software from the twox-hash project (MIT License)
https://github.com/shepmaster/twox-hash
https://github.com/shepmaster/twox-hash

This product includes software developed at
Apache Gluten (https://github.com/apache/incubator-gluten/)
Specifically:
- Optimizer rule to replace SortMergeJoin with ShuffleHashJoin
10 changes: 10 additions & 0 deletions common/src/main/scala/org/apache/comet/CometConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,9 @@ import org.apache.comet.shims.ShimCometConf
*/
object CometConf extends ShimCometConf {

private val TUNING_GUIDE = "For more information, refer to the Comet Tuning " +
"Guide (https://datafusion.apache.org/comet/user-guide/tuning.html)"

/** List of all configs that is used for generating documentation */
val allConfs = new ListBuffer[ConfigEntry[_]]

Expand Down Expand Up @@ -263,6 +266,13 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(false)

val COMET_REPLACE_SMJ: ConfigEntry[Boolean] =
conf(s"$COMET_EXEC_CONFIG_PREFIX.replaceSortMergeJoin")
.doc("Experimental feature to force Spark to replace SortMergeJoin with ShuffledHashJoin " +
s"for improved performance. This feature is not stable yet. $TUNING_GUIDE.")
.booleanConf
.createWithDefault(false)

val COMET_EXEC_SHUFFLE_CODEC: ConfigEntry[String] = conf(
s"$COMET_EXEC_CONFIG_PREFIX.shuffle.codec")
.doc(
Expand Down
1 change: 1 addition & 0 deletions docs/source/contributor-guide/benchmarking.md
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ $SPARK_HOME/bin/spark-submit \
--conf spark.executor.extraClassPath=$COMET_JAR \
--conf spark.plugins=org.apache.spark.CometPlugin \
--conf spark.comet.cast.allowIncompatible=true \
--conf spark.comet.exec.replaceSortMergeJoin=true \
--conf spark.comet.exec.shuffle.enabled=true \
--conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \
tpcbench.py \
Expand Down
1 change: 1 addition & 0 deletions docs/source/user-guide/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ Comet provides the following configuration settings.
| spark.comet.exec.localLimit.enabled | Whether to enable localLimit by default. | true |
| spark.comet.exec.memoryFraction | The fraction of memory from Comet memory overhead that the native memory manager can use for execution. The purpose of this config is to set aside memory for untracked data structures, as well as imprecise size estimation during memory acquisition. Default value is 0.7. | 0.7 |
| spark.comet.exec.project.enabled | Whether to enable project by default. | true |
| spark.comet.exec.replaceSortMergeJoin | Experimental feature to force Spark to replace SortMergeJoin with ShuffledHashJoin for improved performance. This feature is not stable yet. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | false |
| spark.comet.exec.shuffle.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. | zstd |
| spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | true |
| spark.comet.exec.sort.enabled | Whether to enable sort by default. | true |
Expand Down
12 changes: 12 additions & 0 deletions docs/source/user-guide/tuning.md
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,18 @@ Note that there is currently a known issue where this will be inaccurate when us
does not take executor concurrency into account. The tracking issue for this is
https://github.com/apache/datafusion-comet/issues/949.

## Optimizing Joins

Spark often chooses `SortMergeJoin` over `ShuffledHashJoin` for stability reasons. If the build-side of a
`ShuffledHashJoin` is very large then it could lead to OOM in Spark.

Vectorized query engines tend to perform better with `ShuffledHashJoin`, so for best performance it is often preferable
to configure Comet to convert `SortMergeJoin` to `ShuffledHashJoin`. Comet does not yet provide spill-to-disk for
`ShuffledHashJoin` so this could result in OOM. Also, `SortMergeJoin` may still be faster in some cases. It is best
to test with both for your specific workloads.

To configure Comet to convert `SortMergeJoin` to `ShuffledHashJoin`, set `spark.comet.exec.replaceSortMergeJoin=true`.

## Shuffle

Comet provides accelerated shuffle implementations that can be used to improve the performance of your queries.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ import org.apache.comet.CometConf._
import org.apache.comet.CometExplainInfo.getActualPlan
import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos}
import org.apache.comet.parquet.{CometParquetScan, SupportsComet}
import org.apache.comet.rules.RewriteJoin
import org.apache.comet.serde.OperatorOuterClass.Operator
import org.apache.comet.serde.QueryPlanSerde
import org.apache.comet.shims.ShimCometSparkSessionExtensions
Expand Down Expand Up @@ -938,7 +939,15 @@ class CometSparkSessionExtensions
plan
}
} else {
var newPlan = transform(normalizePlan(plan))
val normalizedPlan = if (CometConf.COMET_REPLACE_SMJ.get()) {
normalizePlan(plan).transformUp { case p =>
RewriteJoin.rewrite(p)
}
} else {
normalizePlan(plan)
}

var newPlan = transform(normalizedPlan)

// if the plan cannot be run fully natively then explain why (when appropriate
// config is enabled)
Expand Down
70 changes: 70 additions & 0 deletions spark/src/main/scala/org/apache/comet/rules/RewriteJoin.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.comet.rules

import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper}
import org.apache.spark.sql.catalyst.plans.{JoinType, LeftSemi}
import org.apache.spark.sql.execution.{SortExec, SparkPlan}
import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoinExec}

/**
* Adapted from equivalent rule in Apache Gluten.
*
* This rule replaces [[SortMergeJoinExec]] with [[ShuffledHashJoinExec]].
*/
object RewriteJoin extends JoinSelectionHelper {

private def getBuildSide(joinType: JoinType): Option[BuildSide] = {
if (canBuildShuffledHashJoinRight(joinType)) {
Some(BuildRight)
} else if (canBuildShuffledHashJoinLeft(joinType)) {
Some(BuildLeft)
} else {
None
}
}

private def removeSort(plan: SparkPlan) = plan match {
case _: SortExec => plan.children.head
case _ => plan
}

def rewrite(plan: SparkPlan): SparkPlan = plan match {
case smj: SortMergeJoinExec =>
getBuildSide(smj.joinType) match {
case Some(BuildRight) if smj.joinType == LeftSemi =>
// TODO this was added as a workaround for TPC-DS q14 hanging and needs
// further investigation
plan
case Some(buildSide) =>
ShuffledHashJoinExec(
smj.leftKeys,
smj.rightKeys,
smj.joinType,
buildSide,
smj.condition,
removeSort(smj.left),
removeSort(smj.right),
smj.isSkewJoin)
case _ => plan
}
case _ => plan
}
}

0 comments on commit cb3e977

Please sign in to comment.