Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
6 changes: 6 additions & 0 deletions common/src/main/scala/org/apache/comet/CometConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -263,6 +263,12 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(false)

val COMET_REPLACE_SMJ: ConfigEntry[Boolean] =
conf(s"$COMET_EXEC_CONFIG_PREFIX.replaceSortMergeJoin")
.doc("Whether to replace SortMergeJoin with ShuffledHashJoin for improved performance.")
.booleanConf
.createWithDefault(true)

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/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 | Whether to replace SortMergeJoin with ShuffledHashJoin for improved performance. | true |
| 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
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
66 changes: 66 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,66 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.comet.rules

import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper}
import org.apache.spark.sql.catalyst.plans.JoinType
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(buildSide) =>
ShuffledHashJoinExec(
smj.leftKeys,
smj.rightKeys,
smj.joinType,
buildSide,
smj.condition,
removeSort(smj.left),
removeSort(smj.right),
smj.isSkewJoin)
case _ => plan
}
case _ => plan
}
}
Loading