Skip to content

Commit 40b0707

Browse files
committed
Address comments
1 parent a15af6b commit 40b0707

File tree

10 files changed

+147
-123
lines changed

10 files changed

+147
-123
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -159,14 +159,6 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
159159
children.foldLeft(Option.empty[BaseType]) { (l, r) => l.orElse(r.find(f)) }
160160
}
161161

162-
/**
163-
* Tests whether a predicate holds for all nodes.
164-
* @param p the predicate function to be applied to each node in the tree.
165-
*/
166-
def forall(p: BaseType => Boolean): Boolean = {
167-
p(this) && children.forall(_.forall(p))
168-
}
169-
170162
/**
171163
* Runs the given function on this node and then recursively on [[children]].
172164
* @param f the function to be applied to each node in the tree.

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2574,8 +2574,8 @@ object SQLConf {
25742574
.booleanConf
25752575
.createWithDefault(false)
25762576

2577-
val BUCKETING_COALESCE_ENABLED =
2578-
buildConf("spark.sql.bucketing.coalesce")
2577+
val COALESCE_BUCKET_IN_JOIN_ENABLED =
2578+
buildConf("spark.sql.bucketing.coalesceBucketInJoin.enabled")
25792579
.internal()
25802580
.doc("When true, if two bucketed tables with a different number of buckets are joined, " +
25812581
"the side with a bigger number of buckets will be coalesced to have the same number " +
@@ -2585,6 +2585,16 @@ object SQLConf {
25852585
.booleanConf
25862586
.createWithDefault(false)
25872587

2588+
val COALESCE_BUCKET_IN_JOIN_MAX_NUM_BUCKETS_DIFF =
2589+
buildConf("spark.sql.bucketing.coalesceBucketInJoin.maxNumBucketsDiff")
2590+
.doc("The difference in count of two buckets being coalesced should be less than or " +
2591+
"equal to this value for bucket coalescing to be applied. This configuration only " +
2592+
s"has an effect when '${COALESCE_BUCKET_IN_JOIN_ENABLED.key}' is set to true.")
2593+
.version("3.1.0")
2594+
.intConf
2595+
.checkValue(_ > 0, "The minimum number of partitions must be positive.")
2596+
.createWithDefault(256)
2597+
25882598
/**
25892599
* Holds information about keys that have been deprecated.
25902600
*

sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala

Lines changed: 9 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._
3232
import org.apache.spark.sql.catalyst.plans.QueryPlan
3333
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
3434
import org.apache.spark.sql.catalyst.util.truncatedString
35-
import org.apache.spark.sql.execution.bucketing.InjectBucketHint
35+
import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin
3636
import org.apache.spark.sql.execution.datasources._
3737
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource}
3838
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
@@ -154,7 +154,8 @@ case class RowDataSourceScanExec(
154154
* @param output Output attributes of the scan, including data attributes and partition attributes.
155155
* @param requiredSchema Required schema of the underlying relation, excluding partition columns.
156156
* @param partitionFilters Predicates to use for partition pruning.
157-
* @param optionalBucketSet Bucket ids for bucket pruning
157+
* @param optionalBucketSet Bucket ids for bucket pruning.
158+
* @param optionalCoalescedNumBuckets Coalesced number of buckets.
158159
* @param dataFilters Filters on non-partition columns.
159160
* @param tableIdentifier identifier for the table in the metastore.
160161
*/
@@ -164,6 +165,7 @@ case class FileSourceScanExec(
164165
requiredSchema: StructType,
165166
partitionFilters: Seq[Expression],
166167
optionalBucketSet: Option[BitSet],
168+
optionalCoalescedNumBuckets: Option[Int],
167169
dataFilters: Seq[Expression],
168170
tableIdentifier: Option[TableIdentifier])
169171
extends DataSourceScanExec {
@@ -268,23 +270,6 @@ case class FileSourceScanExec(
268270
}
269271
}
270272

271-
/**
272-
* A bucket can be coalesced if the number of buckets for this relation is divisible
273-
* by the number of buckets on the other side of table for join.
274-
*/
275-
private lazy val coalescedNumBuckets: Option[Int] = {
276-
val joinHintNumBuckets = relation.options.get(InjectBucketHint.JOIN_HINT_NUM_BUCKETS)
277-
if (relation.bucketSpec.isDefined &&
278-
SQLConf.get.getConf(SQLConf.BUCKETING_COALESCE_ENABLED) &&
279-
joinHintNumBuckets.isDefined &&
280-
joinHintNumBuckets.get.toInt < relation.bucketSpec.get.numBuckets &&
281-
relation.bucketSpec.get.numBuckets % joinHintNumBuckets.get.toInt == 0) {
282-
Some(joinHintNumBuckets.get.toInt)
283-
} else {
284-
None
285-
}
286-
}
287-
288273
override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = {
289274
if (bucketedScan) {
290275
// For bucketed columns:
@@ -307,7 +292,7 @@ case class FileSourceScanExec(
307292
// above
308293
val spec = relation.bucketSpec.get
309294
val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n))
310-
val numPartitions = coalescedNumBuckets.getOrElse(spec.numBuckets)
295+
val numPartitions = optionalCoalescedNumBuckets.getOrElse(spec.numBuckets)
311296
val partitioning = HashPartitioning(bucketColumns, numPartitions)
312297
val sortColumns =
313298
spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get)
@@ -329,7 +314,7 @@ case class FileSourceScanExec(
329314
val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1)
330315

331316
// TODO Sort order is currently ignored if buckets are coalesced.
332-
if (singleFilePartitions && coalescedNumBuckets.isEmpty) {
317+
if (singleFilePartitions && optionalCoalescedNumBuckets.isEmpty) {
333318
// TODO Currently Spark does not support writing columns sorting in descending order
334319
// so using Ascending order. This can be fixed in future
335320
sortColumns.map(attribute => SortOrder(attribute, Ascending))
@@ -558,8 +543,8 @@ case class FileSourceScanExec(
558543
filesGroupedToBuckets
559544
}
560545

561-
val filePartitions = if (coalescedNumBuckets.isDefined) {
562-
val newNumBuckets = coalescedNumBuckets.get
546+
val filePartitions = if (optionalCoalescedNumBuckets.isDefined) {
547+
val newNumBuckets = optionalCoalescedNumBuckets.get
563548
logInfo(s"Coalescing to ${newNumBuckets} buckets")
564549
val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % newNumBuckets)
565550
Seq.tabulate(newNumBuckets) { bucketId =>
@@ -625,6 +610,7 @@ case class FileSourceScanExec(
625610
requiredSchema,
626611
QueryPlan.normalizePredicates(partitionFilters, output),
627612
optionalBucketSet,
613+
optionalCoalescedNumBuckets,
628614
QueryPlan.normalizePredicates(dataFilters, output),
629615
None)
630616
}

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer._
2323
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2424
import org.apache.spark.sql.catalyst.rules.Rule
2525
import org.apache.spark.sql.connector.catalog.CatalogManager
26-
import org.apache.spark.sql.execution.bucketing.InjectBucketHint
26+
import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin
2727
import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions
2828
import org.apache.spark.sql.execution.datasources.SchemaPruning
2929
import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown
@@ -45,7 +45,7 @@ class SparkOptimizer(
4545
Batch("PartitionPruning", Once,
4646
PartitionPruning,
4747
OptimizeSubqueries) :+
48-
Batch("Bucketing", Once, InjectBucketHint) :+
48+
Batch("Bucketing", Once, CoalesceBucketInJoin) :+
4949
Batch("Pushdown Filters from PartitionPruning", fixedPoint,
5050
PushDownPredicates) :+
5151
Batch("Cleanup filters that cannot be pushed down", Once,
Lines changed: 110 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,110 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.bucketing
19+
20+
import org.apache.spark.sql.catalyst.catalog.BucketSpec
21+
import org.apache.spark.sql.catalyst.plans.Inner
22+
import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project}
23+
import org.apache.spark.sql.catalyst.rules.Rule
24+
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
25+
import org.apache.spark.sql.internal.SQLConf
26+
27+
/**
28+
* This rule injects a hint if one side of two bucketed tables can be coalesced
29+
* when the two bucketed tables are inner-joined and they differ in the number of buckets.
30+
*/
31+
object CoalesceBucketInJoin extends Rule[LogicalPlan] {
32+
val JOIN_HINT_COALESCED_NUM_BUCKETS: String = "JoinHintCoalescedNumBuckets"
33+
34+
private val sqlConf = SQLConf.get
35+
36+
private def isPlanEligible(plan: LogicalPlan): Boolean = {
37+
def forall(plan: LogicalPlan)(p: LogicalPlan => Boolean): Boolean = {
38+
p(plan) && plan.children.forall(forall(_)(p))
39+
}
40+
41+
forall(plan) {
42+
case _: Filter | _: Project | _: LogicalRelation => true
43+
case _ => false
44+
}
45+
}
46+
47+
private def getBucketSpec(plan: LogicalPlan): Option[BucketSpec] = {
48+
if (isPlanEligible(plan)) {
49+
plan.collectFirst {
50+
case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _)
51+
if r.bucketSpec.nonEmpty && !r.options.contains(JOIN_HINT_COALESCED_NUM_BUCKETS) =>
52+
r.bucketSpec.get
53+
}
54+
} else {
55+
None
56+
}
57+
}
58+
59+
private def mayCoalesce(numBuckets1: Int, numBuckets2: Int): Option[Int] = {
60+
assert(numBuckets1 != numBuckets2)
61+
val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2))
62+
// A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller
63+
// number of buckets because bucket id is calculated by modding the total number of buckets.
64+
if ((large % small == 0) &&
65+
(large - small) <= sqlConf.getConf(SQLConf.COALESCE_BUCKET_IN_JOIN_MAX_NUM_BUCKETS_DIFF)) {
66+
Some(small)
67+
} else {
68+
None
69+
}
70+
}
71+
72+
private def addBucketHint(plan: LogicalPlan, hint: (String, String)): LogicalPlan = {
73+
plan.transformUp {
74+
case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) =>
75+
l.copy(relation = r.copy(options = r.options + hint)(r.sparkSession))
76+
}
77+
}
78+
79+
def apply(plan: LogicalPlan): LogicalPlan = {
80+
if (!sqlConf.getConf(SQLConf.COALESCE_BUCKET_IN_JOIN_ENABLED)) {
81+
return plan
82+
}
83+
84+
plan transform {
85+
case join: Join if join.joinType == Inner =>
86+
val leftBucket = getBucketSpec(join.left)
87+
val rightBucket = getBucketSpec(join.right)
88+
if (leftBucket.isEmpty || rightBucket.isEmpty) {
89+
return plan
90+
}
91+
92+
val leftBucketNumber = leftBucket.get.numBuckets
93+
val rightBucketNumber = rightBucket.get.numBuckets
94+
if (leftBucketNumber == rightBucketNumber) {
95+
return plan
96+
}
97+
98+
mayCoalesce(leftBucketNumber, rightBucketNumber).map { coalescedNumBuckets =>
99+
val hint = JOIN_HINT_COALESCED_NUM_BUCKETS -> coalescedNumBuckets.toString
100+
if (coalescedNumBuckets != leftBucketNumber) {
101+
join.copy(left = addBucketHint(join.left, hint))
102+
} else {
103+
join.copy(right = addBucketHint(join.right, hint))
104+
}
105+
}.getOrElse(join)
106+
107+
case other => other
108+
}
109+
}
110+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala

Lines changed: 0 additions & 78 deletions
This file was deleted.

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._
2525
import org.apache.spark.sql.catalyst.planning.ScanOperation
2626
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2727
import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan}
28+
import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin
2829
import org.apache.spark.util.collection.BitSet
2930

3031
/**
@@ -205,6 +206,7 @@ object FileSourceStrategy extends Strategy with Logging {
205206
outputSchema,
206207
partitionKeyFilters.toSeq,
207208
bucketSet,
209+
fsRelation.options.get(CoalesceBucketInJoin.JOIN_HINT_COALESCED_NUM_BUCKETS).map(_.toInt),
208210
dataFilters,
209211
table.map(_.identifier))
210212

sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -345,7 +345,7 @@ class DataFrameJoinSuite extends QueryTest
345345
}
346346
assert(broadcastExchanges.size == 1)
347347
val tables = broadcastExchanges.head.collect {
348-
case FileSourceScanExec(_, _, _, _, _, _, Some(tableIdent)) => tableIdent
348+
case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent)) => tableIdent
349349
}
350350
assert(tables.size == 1)
351351
assert(tables.head === TableIdentifier(table1Name, Some(dbName)))

sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1311,7 +1311,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
13111311
// need to execute the query before we can examine fs.inputRDDs()
13121312
assert(stripAQEPlan(df.queryExecution.executedPlan) match {
13131313
case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(
1314-
fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _)))) =>
1314+
fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _)))) =>
13151315
partitionFilters.exists(ExecSubqueryExpression.hasSubquery) &&
13161316
fs.inputRDDs().forall(
13171317
_.asInstanceOf[FileScanRDD].filePartitions.forall(

0 commit comments

Comments
 (0)