From 94720e00d222939085a7a5fd584608b8cfb2fc47 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 4 Apr 2020 15:55:07 -0700 Subject: [PATCH 01/21] initial commit --- .../spark/sql/catalyst/trees/TreeNode.scala | 8 ++ .../apache/spark/sql/internal/SQLConf.scala | 11 +++ .../sql/execution/DataSourceScanExec.scala | 40 +++++++++- .../spark/sql/execution/SparkOptimizer.scala | 2 + .../bucketing/InjectBucketHint.scala | 80 +++++++++++++++++++ .../spark/sql/sources/BucketedReadSuite.scala | 53 ++++++++++-- 6 files changed, 185 insertions(+), 9 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index c4a106702a515..abe03e8f53fad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -159,6 +159,14 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { children.foldLeft(Option.empty[BaseType]) { (l, r) => l.orElse(r.find(f)) } } + /** + * Tests whether a predicate holds for all nodes. + * @param p the predicate function to be applied to each node in the tree. + */ + def forall(p: BaseType => Boolean): Boolean = { + p(this) && children.forall(_.forall(p)) + } + /** * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c8a7b234520cf..f96b83518e52e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2574,6 +2574,17 @@ object SQLConf { .booleanConf .createWithDefault(false) + val BUCKETING_COALESCE_ENABLED = + buildConf("spark.sql.bucketing.coalesce") + .internal() + .doc("When true, if two bucketed tables with a different number of buckets are joined, " + + "the side with a bigger number of buckets will be coalesced to have the same number " + + "of buckets as the other side. This bucket coalescing can happen only when the bigger " + + "number of buckets is divisible by the smaller number of buckets.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 90b08ea558044..1b8d94f8b02bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.execution.bucketing.InjectBucketHint import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} @@ -267,6 +268,23 @@ case class FileSourceScanExec( } } + /** + * A bucket can be coalesced if the number of buckets for this relation is divisible + * by the number of buckets on the other side of table for join. + */ + private lazy val coalescedNumBuckets: Option[Int] = { + val joinHintNumBuckets = relation.options.get(InjectBucketHint.JOIN_HINT_NUM_BUCKETS) + if (relation.bucketSpec.isDefined && + SQLConf.get.getConf(SQLConf.BUCKETING_COALESCE_ENABLED) && + joinHintNumBuckets.isDefined && + joinHintNumBuckets.get.toInt < relation.bucketSpec.get.numBuckets && + relation.bucketSpec.get.numBuckets % joinHintNumBuckets.get.toInt == 0) { + Some(joinHintNumBuckets.get.toInt) + } else { + None + } + } + override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { if (bucketedScan) { // For bucketed columns: @@ -289,7 +307,8 @@ case class FileSourceScanExec( // above val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - val partitioning = HashPartitioning(bucketColumns, spec.numBuckets) + val numPartitions = coalescedNumBuckets.getOrElse(spec.numBuckets) + val partitioning = HashPartitioning(bucketColumns, numPartitions) val sortColumns = spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) val shouldCalculateSortOrder = @@ -309,7 +328,8 @@ case class FileSourceScanExec( files.map(_.getPath.getName).groupBy(file => BucketingUtils.getBucketId(file)) val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) - if (singleFilePartitions) { + // TODO Sort order is currently ignored if buckets are coalesced. + if (singleFilePartitions && coalescedNumBuckets.isEmpty) { // TODO Currently Spark does not support writing columns sorting in descending order // so using Ascending order. This can be fixed in future sortColumns.map(attribute => SortOrder(attribute, Ascending)) @@ -538,8 +558,20 @@ case class FileSourceScanExec( filesGroupedToBuckets } - val filePartitions = Seq.tabulate(bucketSpec.numBuckets) { bucketId => - FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + val filePartitions = if (coalescedNumBuckets.isDefined) { + val newNumBuckets = coalescedNumBuckets.get + logInfo(s"Coalescing to ${newNumBuckets} buckets") + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % newNumBuckets) + Seq.tabulate(newNumBuckets) { bucketId => + val partitionedFiles = coalescedBuckets.get(bucketId).map { + _.values.flatten.toArray + }.getOrElse(Array.empty) + FilePartition(bucketId, partitionedFiles) + } + } else { + Seq.tabulate(bucketSpec.numBuckets) { bucketId => + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + } } new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 33b86a2b5340c..b0b25924c7ce7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.execution.bucketing.InjectBucketHint import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown @@ -44,6 +45,7 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ + Batch("Bucketing", Once, InjectBucketHint) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala new file mode 100644 index 0000000000000..574f8cf78e515 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala @@ -0,0 +1,80 @@ +/* + * 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.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} + +/** + * This rule injects the bucket related hints and supports the following scenarios: + * - If the two bucketed tables are inner-joined and they differ in the number of buckets, + * the number of buckets for the other table of join is injected as a hint. + */ +object InjectBucketHint extends Rule[LogicalPlan] { + val JOIN_HINT_NUM_BUCKETS: String = "JoinHintNumBuckets" + + private def isPlanEligible(plan: LogicalPlan): Boolean = { + plan.forall { + case _: Filter => true + case _: Project => true + case _: LogicalRelation => true + case _ => false + } + } + + private def getBucketSpec(plan: LogicalPlan): Option[BucketSpec] = { + if (isPlanEligible(plan)) { + plan.collectFirst { + case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _) + if r.bucketSpec.nonEmpty && !r.options.contains(JOIN_HINT_NUM_BUCKETS) => + r.bucketSpec.get + } + } else { + None + } + } + + def apply(plan: LogicalPlan): LogicalPlan = { + plan transform { + case join : Join if join.joinType == Inner => + val leftBucket = getBucketSpec(join.left) + val rightBucket = getBucketSpec(join.right) + if (leftBucket.isEmpty || + rightBucket.isEmpty || + leftBucket.get.numBuckets == rightBucket.get.numBuckets) { + return plan + } + + def addBucketHint(subPlan: LogicalPlan, numBuckets: Int): LogicalPlan = { + subPlan.transformUp { + case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => + val newOption = JOIN_HINT_NUM_BUCKETS -> numBuckets.toString + l.copy(relation = r.copy(options = r.options + newOption)(r.sparkSession)) + } + } + + join.copy( + left = addBucketHint(join.left, rightBucket.get.numBuckets), + right = addBucketHint(join.right, leftBucket.get.numBuckets)) + case other => other + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 1d8303b9e7750..58106dd59753b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -332,7 +332,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { bucketSpec: Option[BucketSpec], numPartitions: Int = 10, expectedShuffle: Boolean = true, - expectedSort: Boolean = true) + expectedSort: Boolean = true, + expectedNumOutputPartitions: Option[Int] = None) /** * A helper method to test the bucket read functionality using join. It will save `df1` and `df2` @@ -345,10 +346,18 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { bucketedTableTestSpecRight: BucketedTableTestSpec, joinType: String = "inner", joinCondition: (DataFrame, DataFrame) => Column): Unit = { - val BucketedTableTestSpec(bucketSpecLeft, numPartitionsLeft, shuffleLeft, sortLeft) = - bucketedTableTestSpecLeft - val BucketedTableTestSpec(bucketSpecRight, numPartitionsRight, shuffleRight, sortRight) = - bucketedTableTestSpecRight + val BucketedTableTestSpec( + bucketSpecLeft, + numPartitionsLeft, + shuffleLeft, + sortLeft, + numOutputPartitionsLeft) = bucketedTableTestSpecLeft + val BucketedTableTestSpec( + bucketSpecRight, + numPartitionsRight, + shuffleRight, + sortRight, + numOutputPartitionsRight) = bucketedTableTestSpecRight withTable("bucketed_table1", "bucketed_table2") { def withBucket( @@ -413,6 +422,16 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { assert( joinOperator.right.find(_.isInstanceOf[SortExec]).isDefined == sortRight, s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") + + // check the output partitioning + if (numOutputPartitionsLeft.isDefined) { + assert(joinOperator.left.outputPartitioning.numPartitions === + numOutputPartitionsLeft.get) + } + if (numOutputPartitionsRight.isDefined) { + assert(joinOperator.right.outputPartitioning.numPartitions === + numOutputPartitionsRight.get) + } } } } @@ -855,4 +874,28 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } } + + test("bucket coalescing eliminates shuffle") { + withSQLConf(SQLConf.BUCKETING_COALESCE_ENABLED.key -> "true") { + // Left side will be coalesced to have 4 output partitions. + // Currently, sort will be introduced for the side that is coalesced. + val bucketedTableTestSpecLeft = BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = true, + expectedNumOutputPartitions = Some(4)) + val bucketedTableTestSpecRight = BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = false, + expectedNumOutputPartitions = Some(4)) + + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j"))) + } + } } From a15af6b3c55d76fb30614b03ec2b5c1f093e2ef2 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 4 Apr 2020 16:57:29 -0700 Subject: [PATCH 02/21] minor improvement --- .../spark/sql/execution/bucketing/InjectBucketHint.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala index 574f8cf78e515..de36686386c61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala @@ -33,9 +33,7 @@ object InjectBucketHint extends Rule[LogicalPlan] { private def isPlanEligible(plan: LogicalPlan): Boolean = { plan.forall { - case _: Filter => true - case _: Project => true - case _: LogicalRelation => true + case _: Filter | _: Project | _: LogicalRelation => true case _ => false } } From 40b07073949e41fe275a38f7a367b789ed279afb Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 13 Apr 2020 16:59:31 -0700 Subject: [PATCH 03/21] Address comments --- .../spark/sql/catalyst/trees/TreeNode.scala | 8 -- .../apache/spark/sql/internal/SQLConf.scala | 14 ++- .../sql/execution/DataSourceScanExec.scala | 32 ++--- .../spark/sql/execution/SparkOptimizer.scala | 4 +- .../bucketing/CoalesceBucketInJoin.scala | 110 ++++++++++++++++++ .../bucketing/InjectBucketHint.scala | 78 ------------- .../datasources/FileSourceStrategy.scala | 2 + .../apache/spark/sql/DataFrameJoinSuite.scala | 2 +- .../org/apache/spark/sql/SubquerySuite.scala | 2 +- .../spark/sql/sources/BucketedReadSuite.scala | 18 +-- 10 files changed, 147 insertions(+), 123 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index abe03e8f53fad..c4a106702a515 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -159,14 +159,6 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { children.foldLeft(Option.empty[BaseType]) { (l, r) => l.orElse(r.find(f)) } } - /** - * Tests whether a predicate holds for all nodes. - * @param p the predicate function to be applied to each node in the tree. - */ - def forall(p: BaseType => Boolean): Boolean = { - p(this) && children.forall(_.forall(p)) - } - /** * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f96b83518e52e..d6d05814862b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2574,8 +2574,8 @@ object SQLConf { .booleanConf .createWithDefault(false) - val BUCKETING_COALESCE_ENABLED = - buildConf("spark.sql.bucketing.coalesce") + val COALESCE_BUCKET_IN_JOIN_ENABLED = + buildConf("spark.sql.bucketing.coalesceBucketInJoin.enabled") .internal() .doc("When true, if two bucketed tables with a different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + @@ -2585,6 +2585,16 @@ object SQLConf { .booleanConf .createWithDefault(false) + val COALESCE_BUCKET_IN_JOIN_MAX_NUM_BUCKETS_DIFF = + buildConf("spark.sql.bucketing.coalesceBucketInJoin.maxNumBucketsDiff") + .doc("The difference in count of two buckets being coalesced should be less than or " + + "equal to this value for bucket coalescing to be applied. This configuration only " + + s"has an effect when '${COALESCE_BUCKET_IN_JOIN_ENABLED.key}' is set to true.") + .version("3.1.0") + .intConf + .checkValue(_ > 0, "The minimum number of partitions must be positive.") + .createWithDefault(256) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 1b8d94f8b02bf..48aa7c4c3bcfc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.bucketing.InjectBucketHint +import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} @@ -154,7 +154,8 @@ case class RowDataSourceScanExec( * @param output Output attributes of the scan, including data attributes and partition attributes. * @param requiredSchema Required schema of the underlying relation, excluding partition columns. * @param partitionFilters Predicates to use for partition pruning. - * @param optionalBucketSet Bucket ids for bucket pruning + * @param optionalBucketSet Bucket ids for bucket pruning. + * @param optionalCoalescedNumBuckets Coalesced number of buckets. * @param dataFilters Filters on non-partition columns. * @param tableIdentifier identifier for the table in the metastore. */ @@ -164,6 +165,7 @@ case class FileSourceScanExec( requiredSchema: StructType, partitionFilters: Seq[Expression], optionalBucketSet: Option[BitSet], + optionalCoalescedNumBuckets: Option[Int], dataFilters: Seq[Expression], tableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec { @@ -268,23 +270,6 @@ case class FileSourceScanExec( } } - /** - * A bucket can be coalesced if the number of buckets for this relation is divisible - * by the number of buckets on the other side of table for join. - */ - private lazy val coalescedNumBuckets: Option[Int] = { - val joinHintNumBuckets = relation.options.get(InjectBucketHint.JOIN_HINT_NUM_BUCKETS) - if (relation.bucketSpec.isDefined && - SQLConf.get.getConf(SQLConf.BUCKETING_COALESCE_ENABLED) && - joinHintNumBuckets.isDefined && - joinHintNumBuckets.get.toInt < relation.bucketSpec.get.numBuckets && - relation.bucketSpec.get.numBuckets % joinHintNumBuckets.get.toInt == 0) { - Some(joinHintNumBuckets.get.toInt) - } else { - None - } - } - override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { if (bucketedScan) { // For bucketed columns: @@ -307,7 +292,7 @@ case class FileSourceScanExec( // above val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - val numPartitions = coalescedNumBuckets.getOrElse(spec.numBuckets) + val numPartitions = optionalCoalescedNumBuckets.getOrElse(spec.numBuckets) val partitioning = HashPartitioning(bucketColumns, numPartitions) val sortColumns = spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) @@ -329,7 +314,7 @@ case class FileSourceScanExec( val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) // TODO Sort order is currently ignored if buckets are coalesced. - if (singleFilePartitions && coalescedNumBuckets.isEmpty) { + if (singleFilePartitions && optionalCoalescedNumBuckets.isEmpty) { // TODO Currently Spark does not support writing columns sorting in descending order // so using Ascending order. This can be fixed in future sortColumns.map(attribute => SortOrder(attribute, Ascending)) @@ -558,8 +543,8 @@ case class FileSourceScanExec( filesGroupedToBuckets } - val filePartitions = if (coalescedNumBuckets.isDefined) { - val newNumBuckets = coalescedNumBuckets.get + val filePartitions = if (optionalCoalescedNumBuckets.isDefined) { + val newNumBuckets = optionalCoalescedNumBuckets.get logInfo(s"Coalescing to ${newNumBuckets} buckets") val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % newNumBuckets) Seq.tabulate(newNumBuckets) { bucketId => @@ -625,6 +610,7 @@ case class FileSourceScanExec( requiredSchema, QueryPlan.normalizePredicates(partitionFilters, output), optionalBucketSet, + optionalCoalescedNumBuckets, QueryPlan.normalizePredicates(dataFilters, output), None) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index b0b25924c7ce7..f6d9248d97531 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.bucketing.InjectBucketHint +import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown @@ -45,7 +45,7 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Bucketing", Once, InjectBucketHint) :+ + Batch("Bucketing", Once, CoalesceBucketInJoin) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala new file mode 100644 index 0000000000000..f380dbe746beb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala @@ -0,0 +1,110 @@ +/* + * 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.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.internal.SQLConf + +/** + * This rule injects a hint if one side of two bucketed tables can be coalesced + * when the two bucketed tables are inner-joined and they differ in the number of buckets. + */ +object CoalesceBucketInJoin extends Rule[LogicalPlan] { + val JOIN_HINT_COALESCED_NUM_BUCKETS: String = "JoinHintCoalescedNumBuckets" + + private val sqlConf = SQLConf.get + + private def isPlanEligible(plan: LogicalPlan): Boolean = { + def forall(plan: LogicalPlan)(p: LogicalPlan => Boolean): Boolean = { + p(plan) && plan.children.forall(forall(_)(p)) + } + + forall(plan) { + case _: Filter | _: Project | _: LogicalRelation => true + case _ => false + } + } + + private def getBucketSpec(plan: LogicalPlan): Option[BucketSpec] = { + if (isPlanEligible(plan)) { + plan.collectFirst { + case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _) + if r.bucketSpec.nonEmpty && !r.options.contains(JOIN_HINT_COALESCED_NUM_BUCKETS) => + r.bucketSpec.get + } + } else { + None + } + } + + private def mayCoalesce(numBuckets1: Int, numBuckets2: Int): Option[Int] = { + assert(numBuckets1 != numBuckets2) + val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) + // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller + // number of buckets because bucket id is calculated by modding the total number of buckets. + if ((large % small == 0) && + (large - small) <= sqlConf.getConf(SQLConf.COALESCE_BUCKET_IN_JOIN_MAX_NUM_BUCKETS_DIFF)) { + Some(small) + } else { + None + } + } + + private def addBucketHint(plan: LogicalPlan, hint: (String, String)): LogicalPlan = { + plan.transformUp { + case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => + l.copy(relation = r.copy(options = r.options + hint)(r.sparkSession)) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (!sqlConf.getConf(SQLConf.COALESCE_BUCKET_IN_JOIN_ENABLED)) { + return plan + } + + plan transform { + case join: Join if join.joinType == Inner => + val leftBucket = getBucketSpec(join.left) + val rightBucket = getBucketSpec(join.right) + if (leftBucket.isEmpty || rightBucket.isEmpty) { + return plan + } + + val leftBucketNumber = leftBucket.get.numBuckets + val rightBucketNumber = rightBucket.get.numBuckets + if (leftBucketNumber == rightBucketNumber) { + return plan + } + + mayCoalesce(leftBucketNumber, rightBucketNumber).map { coalescedNumBuckets => + val hint = JOIN_HINT_COALESCED_NUM_BUCKETS -> coalescedNumBuckets.toString + if (coalescedNumBuckets != leftBucketNumber) { + join.copy(left = addBucketHint(join.left, hint)) + } else { + join.copy(right = addBucketHint(join.right, hint)) + } + }.getOrElse(join) + + case other => other + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala deleted file mode 100644 index de36686386c61..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/InjectBucketHint.scala +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.spark.sql.execution.bucketing - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} - -/** - * This rule injects the bucket related hints and supports the following scenarios: - * - If the two bucketed tables are inner-joined and they differ in the number of buckets, - * the number of buckets for the other table of join is injected as a hint. - */ -object InjectBucketHint extends Rule[LogicalPlan] { - val JOIN_HINT_NUM_BUCKETS: String = "JoinHintNumBuckets" - - private def isPlanEligible(plan: LogicalPlan): Boolean = { - plan.forall { - case _: Filter | _: Project | _: LogicalRelation => true - case _ => false - } - } - - private def getBucketSpec(plan: LogicalPlan): Option[BucketSpec] = { - if (isPlanEligible(plan)) { - plan.collectFirst { - case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _) - if r.bucketSpec.nonEmpty && !r.options.contains(JOIN_HINT_NUM_BUCKETS) => - r.bucketSpec.get - } - } else { - None - } - } - - def apply(plan: LogicalPlan): LogicalPlan = { - plan transform { - case join : Join if join.joinType == Inner => - val leftBucket = getBucketSpec(join.left) - val rightBucket = getBucketSpec(join.right) - if (leftBucket.isEmpty || - rightBucket.isEmpty || - leftBucket.get.numBuckets == rightBucket.get.numBuckets) { - return plan - } - - def addBucketHint(subPlan: LogicalPlan, numBuckets: Int): LogicalPlan = { - subPlan.transformUp { - case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => - val newOption = JOIN_HINT_NUM_BUCKETS -> numBuckets.toString - l.copy(relation = r.copy(options = r.options + newOption)(r.sparkSession)) - } - } - - join.copy( - left = addBucketHint(join.left, rightBucket.get.numBuckets), - right = addBucketHint(join.right, leftBucket.get.numBuckets)) - case other => other - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index f45495121a980..37407f5fb9607 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin import org.apache.spark.util.collection.BitSet /** @@ -205,6 +206,7 @@ object FileSourceStrategy extends Strategy with Logging { outputSchema, partitionKeyFilters.toSeq, bucketSet, + fsRelation.options.get(CoalesceBucketInJoin.JOIN_HINT_COALESCED_NUM_BUCKETS).map(_.toInt), dataFilters, table.map(_.identifier)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 6b772e53ac184..0b4f43b72366b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -345,7 +345,7 @@ class DataFrameJoinSuite extends QueryTest } assert(broadcastExchanges.size == 1) val tables = broadcastExchanges.head.collect { - case FileSourceScanExec(_, _, _, _, _, _, Some(tableIdent)) => tableIdent + case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent)) => tableIdent } assert(tables.size == 1) assert(tables.head === TableIdentifier(table1Name, Some(dbName))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index ff8f94c68c5ee..40d90df3655a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1311,7 +1311,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark // need to execute the query before we can examine fs.inputRDDs() assert(stripAQEPlan(df.queryExecution.executedPlan) match { case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( - fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _)))) => + fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _)))) => partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 58106dd59753b..be750ea84d38d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -876,26 +876,28 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } test("bucket coalescing eliminates shuffle") { - withSQLConf(SQLConf.BUCKETING_COALESCE_ENABLED.key -> "true") { - // Left side will be coalesced to have 4 output partitions. + withSQLConf(SQLConf.COALESCE_BUCKET_IN_JOIN_ENABLED.key -> "true") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. // Currently, sort will be introduced for the side that is coalesced. - val bucketedTableTestSpecLeft = BucketedTableTestSpec( + val testSpec1 = BucketedTableTestSpec( Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), numPartitions = 1, expectedShuffle = false, expectedSort = true, expectedNumOutputPartitions = Some(4)) - val bucketedTableTestSpecRight = BucketedTableTestSpec( + val testSpec2 = BucketedTableTestSpec( Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), numPartitions = 1, expectedShuffle = false, expectedSort = false, expectedNumOutputPartitions = Some(4)) - testBucketing( - bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, - bucketedTableTestSpecRight = bucketedTableTestSpecRight, - joinCondition = joinCondition(Seq("i", "j"))) + Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => + testBucketing( + bucketedTableTestSpecLeft = specs._1, + bucketedTableTestSpecRight = specs._2, + joinCondition = joinCondition(Seq("i", "j"))) + } } } } From 0b9abee89e1366fd4952c462124f4b520c399a6f Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 20 Apr 2020 19:30:29 -0700 Subject: [PATCH 04/21] address comments --- .../apache/spark/sql/internal/SQLConf.scala | 18 +++-- .../sql/execution/DataSourceScanExec.scala | 25 ++++--- .../spark/sql/execution/SparkOptimizer.scala | 4 +- ...Join.scala => CoalesceBucketsInJoin.scala} | 71 ++++++++++--------- .../execution/bucketing/CoalseceBuckets.scala | 34 +++++++++ .../datasources/FileSourceStrategy.scala | 27 +++++-- .../spark/sql/sources/BucketedReadSuite.scala | 2 +- 7 files changed, 119 insertions(+), 62 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/{CoalesceBucketInJoin.scala => CoalesceBucketsInJoin.scala} (62%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalseceBuckets.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4e0f3be5c0a57..d28ad68c0b642 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2575,9 +2575,8 @@ object SQLConf { .booleanConf .createWithDefault(false) - val COALESCE_BUCKET_IN_JOIN_ENABLED = - buildConf("spark.sql.bucketing.coalesceBucketInJoin.enabled") - .internal() + val COALESCE_BUCKETS_IN_JOIN_ENABLED = + buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled") .doc("When true, if two bucketed tables with a different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + "of buckets as the other side. This bucket coalescing can happen only when the bigger " + @@ -2586,14 +2585,14 @@ object SQLConf { .booleanConf .createWithDefault(false) - val COALESCE_BUCKET_IN_JOIN_MAX_NUM_BUCKETS_DIFF = - buildConf("spark.sql.bucketing.coalesceBucketInJoin.maxNumBucketsDiff") + val COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF = + buildConf("spark.sql.bucketing.coalesceBucketsInJoin.maxNumBucketsDiff") .doc("The difference in count of two buckets being coalesced should be less than or " + "equal to this value for bucket coalescing to be applied. This configuration only " + - s"has an effect when '${COALESCE_BUCKET_IN_JOIN_ENABLED.key}' is set to true.") + s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set to true.") .version("3.1.0") .intConf - .checkValue(_ > 0, "The minimum number of partitions must be positive.") + .checkValue(_ > 0, "The difference must be positive.") .createWithDefault(256) /** @@ -2892,6 +2891,11 @@ class SQLConf extends Serializable with Logging { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } + def coalesceBucketsInJoinEnabled: Boolean = getConf(COALESCE_BUCKETS_IN_JOIN_ENABLED) + + def coalesceBucketsInJoinMaxNumBucketsDiff: Int = + getConf(COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF) + /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 48aa7c4c3bcfc..f79f26b6d6725 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} @@ -155,7 +155,7 @@ case class RowDataSourceScanExec( * @param requiredSchema Required schema of the underlying relation, excluding partition columns. * @param partitionFilters Predicates to use for partition pruning. * @param optionalBucketSet Bucket ids for bucket pruning. - * @param optionalCoalescedNumBuckets Coalesced number of buckets. + * @param optionalNumCoalescedBuckets Number of coalesced buckets. * @param dataFilters Filters on non-partition columns. * @param tableIdentifier identifier for the table in the metastore. */ @@ -165,7 +165,7 @@ case class FileSourceScanExec( requiredSchema: StructType, partitionFilters: Seq[Expression], optionalBucketSet: Option[BitSet], - optionalCoalescedNumBuckets: Option[Int], + optionalNumCoalescedBuckets: Option[Int], dataFilters: Seq[Expression], tableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec { @@ -292,7 +292,7 @@ case class FileSourceScanExec( // above val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - val numPartitions = optionalCoalescedNumBuckets.getOrElse(spec.numBuckets) + val numPartitions = optionalNumCoalescedBuckets.getOrElse(spec.numBuckets) val partitioning = HashPartitioning(bucketColumns, numPartitions) val sortColumns = spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) @@ -313,8 +313,8 @@ case class FileSourceScanExec( files.map(_.getPath.getName).groupBy(file => BucketingUtils.getBucketId(file)) val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) - // TODO Sort order is currently ignored if buckets are coalesced. - if (singleFilePartitions && optionalCoalescedNumBuckets.isEmpty) { + // TODO SPARK-24528 Sort order is currently ignored if buckets are coalesced. + if (singleFilePartitions && optionalNumCoalescedBuckets.isEmpty) { // TODO Currently Spark does not support writing columns sorting in descending order // so using Ascending order. This can be fixed in future sortColumns.map(attribute => SortOrder(attribute, Ascending)) @@ -543,17 +543,16 @@ case class FileSourceScanExec( filesGroupedToBuckets } - val filePartitions = if (optionalCoalescedNumBuckets.isDefined) { - val newNumBuckets = optionalCoalescedNumBuckets.get - logInfo(s"Coalescing to ${newNumBuckets} buckets") - val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % newNumBuckets) - Seq.tabulate(newNumBuckets) { bucketId => + val filePartitions = optionalNumCoalescedBuckets.map { numCoalescedBuckets => + logInfo(s"Coalescing to ${numCoalescedBuckets} buckets") + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) + Seq.tabulate(numCoalescedBuckets) { bucketId => val partitionedFiles = coalescedBuckets.get(bucketId).map { _.values.flatten.toArray }.getOrElse(Array.empty) FilePartition(bucketId, partitionedFiles) } - } else { + } getOrElse { Seq.tabulate(bucketSpec.numBuckets) { bucketId => FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) } @@ -610,7 +609,7 @@ case class FileSourceScanExec( requiredSchema, QueryPlan.normalizePredicates(partitionFilters, output), optionalBucketSet, - optionalCoalescedNumBuckets, + optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), None) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index f6d9248d97531..12878cc7c3ada 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown @@ -45,7 +45,7 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Bucketing", Once, CoalesceBucketInJoin) :+ + Batch("Bucketing", Once, CoalesceBucketsInJoin) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala similarity index 62% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala index f380dbe746beb..eb6d46c9468f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketInJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala @@ -28,9 +28,7 @@ import org.apache.spark.sql.internal.SQLConf * This rule injects a hint if one side of two bucketed tables can be coalesced * when the two bucketed tables are inner-joined and they differ in the number of buckets. */ -object CoalesceBucketInJoin extends Rule[LogicalPlan] { - val JOIN_HINT_COALESCED_NUM_BUCKETS: String = "JoinHintCoalescedNumBuckets" - +object CoalesceBucketsInJoin extends Rule[LogicalPlan] { private val sqlConf = SQLConf.get private def isPlanEligible(plan: LogicalPlan): Boolean = { @@ -47,8 +45,7 @@ object CoalesceBucketInJoin extends Rule[LogicalPlan] { private def getBucketSpec(plan: LogicalPlan): Option[BucketSpec] = { if (isPlanEligible(plan)) { plan.collectFirst { - case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _) - if r.bucketSpec.nonEmpty && !r.options.contains(JOIN_HINT_COALESCED_NUM_BUCKETS) => + case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _) if r.bucketSpec.nonEmpty => r.bucketSpec.get } } else { @@ -62,49 +59,53 @@ object CoalesceBucketInJoin extends Rule[LogicalPlan] { // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller // number of buckets because bucket id is calculated by modding the total number of buckets. if ((large % small == 0) && - (large - small) <= sqlConf.getConf(SQLConf.COALESCE_BUCKET_IN_JOIN_MAX_NUM_BUCKETS_DIFF)) { + ((large - small) <= sqlConf.coalesceBucketsInJoinMaxNumBucketsDiff)) { Some(small) } else { None } } - private def addBucketHint(plan: LogicalPlan, hint: (String, String)): LogicalPlan = { + private def addCoalesceBuckets(plan: LogicalPlan, numCoalescedBuckets: Int): LogicalPlan = { plan.transformUp { - case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => - l.copy(relation = r.copy(options = r.options + hint)(r.sparkSession)) + case l @ LogicalRelation(_: HadoopFsRelation, _, _, _) => + CoalesceBuckets(numCoalescedBuckets, l) } } - def apply(plan: LogicalPlan): LogicalPlan = { - if (!sqlConf.getConf(SQLConf.COALESCE_BUCKET_IN_JOIN_ENABLED)) { - return plan - } - - plan transform { - case join: Join if join.joinType == Inner => - val leftBucket = getBucketSpec(join.left) - val rightBucket = getBucketSpec(join.right) - if (leftBucket.isEmpty || rightBucket.isEmpty) { - return plan - } - - val leftBucketNumber = leftBucket.get.numBuckets - val rightBucketNumber = rightBucket.get.numBuckets - if (leftBucketNumber == rightBucketNumber) { - return plan - } - - mayCoalesce(leftBucketNumber, rightBucketNumber).map { coalescedNumBuckets => - val hint = JOIN_HINT_COALESCED_NUM_BUCKETS -> coalescedNumBuckets.toString - if (coalescedNumBuckets != leftBucketNumber) { - join.copy(left = addBucketHint(join.left, hint)) + object ExtractJoinWithBuckets { + def unapply(plan: LogicalPlan): Option[(Join, Int, Int)] = { + plan match { + case join: Join => + val leftBucket = getBucketSpec(join.left) + val rightBucket = getBucketSpec(join.right) + if (leftBucket.isDefined && rightBucket.isDefined) { + Some(join, leftBucket.get.numBuckets, rightBucket.get.numBuckets) } else { - join.copy(right = addBucketHint(join.right, hint)) + None } - }.getOrElse(join) + case _ => None + } + } + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (sqlConf.coalesceBucketsInJoinEnabled) { + plan transform { + case ExtractJoinWithBuckets(join, numLeftBuckets, numRightBuckets) + if numLeftBuckets != numRightBuckets => + mayCoalesce(numLeftBuckets, numRightBuckets).map { numCoalescedBuckets => + if (numCoalescedBuckets != numLeftBuckets) { + join.copy(left = addCoalesceBuckets(join.left, numCoalescedBuckets)) + } else { + join.copy(right = addCoalesceBuckets(join.right, numCoalescedBuckets)) + } + }.getOrElse(join) - case other => other + case other => other + } + } else { + plan } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalseceBuckets.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalseceBuckets.scala new file mode 100644 index 0000000000000..e1eabd792ffc8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalseceBuckets.scala @@ -0,0 +1,34 @@ +/* + * 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.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.UnaryNode +import org.apache.spark.sql.execution.datasources.LogicalRelation + +/** + * Wraps `LogicalRelation` to provide the number of buckets for coalescing. + */ +case class CoalesceBuckets( + numCoalescedBuckets: Int, + child: LogicalRelation) extends UnaryNode { + require(numCoalescedBuckets > 0, + s"Number of coalesced buckets ($numCoalescedBuckets) must be positive.") + + override def output: Seq[Attribute] = child.output +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 37407f5fb9607..96972f9fbb81f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} -import org.apache.spark.sql.execution.bucketing.CoalesceBucketInJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBuckets import org.apache.spark.util.collection.BitSet /** @@ -138,8 +138,8 @@ object FileSourceStrategy extends Strategy with Logging { } def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ScanOperation(projects, filters, - l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _)) => + case ScanOperationWithCoalescedBuckets(projects, filters, + l@LogicalRelation(fsRelation: HadoopFsRelation, _, table, _), numCoalescedBuckets) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read @@ -206,7 +206,7 @@ object FileSourceStrategy extends Strategy with Logging { outputSchema, partitionKeyFilters.toSeq, bucketSet, - fsRelation.options.get(CoalesceBucketInJoin.JOIN_HINT_COALESCED_NUM_BUCKETS).map(_.toInt), + numCoalescedBuckets, dataFilters, table.map(_.identifier)) @@ -223,3 +223,22 @@ object FileSourceStrategy extends Strategy with Logging { case _ => Nil } } + +/** + * Extractor that handles `CoalesceBuckets` in the child plan extracted from `ScanOperation`. + */ +object ScanOperationWithCoalescedBuckets { + type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan, Option[Int]) + + def unapply(plan: LogicalPlan): Option[ReturnType] = { + plan match { + case ScanOperation(projects, filters, child) => child match { + case c: CoalesceBuckets => + Some(projects, filters, c.child, Some(c.numCoalescedBuckets)) + case _ => + Some(projects, filters, child, None) + } + case _ => None + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 7a85ebdc36a79..e5109614b920e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -877,7 +877,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } test("bucket coalescing eliminates shuffle") { - withSQLConf(SQLConf.COALESCE_BUCKET_IN_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. // Currently, sort will be introduced for the side that is coalesced. val testSpec1 = BucketedTableTestSpec( From c93322bd89f7955fbc63d4a8ca61090b922fc60e Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 20 Apr 2020 19:33:37 -0700 Subject: [PATCH 05/21] address comment --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d28ad68c0b642..47f80fefeb0a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2577,7 +2577,7 @@ object SQLConf { val COALESCE_BUCKETS_IN_JOIN_ENABLED = buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled") - .doc("When true, if two bucketed tables with a different number of buckets are joined, " + + .doc("When true, if two bucketed tables with the different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + "of buckets as the other side. This bucket coalescing can happen only when the bigger " + "number of buckets is divisible by the smaller number of buckets.") From 347c2de91ef97da1a783b53622b19662f4044e60 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 20 Apr 2020 19:38:37 -0700 Subject: [PATCH 06/21] style change --- .../spark/sql/execution/datasources/FileSourceStrategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 96972f9fbb81f..4b2e636062aae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -139,7 +139,7 @@ object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ScanOperationWithCoalescedBuckets(projects, filters, - l@LogicalRelation(fsRelation: HadoopFsRelation, _, table, _), numCoalescedBuckets) => + l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _), numCoalescedBuckets) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read From c78e2a23a20e602f4b17446f31d1921bd5a6cfbc Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 23 Apr 2020 20:34:27 -0700 Subject: [PATCH 07/21] add unit tests --- .../spark/sql/sources/BucketedReadSuite.scala | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index e5109614b920e..0b67537b37326 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -901,4 +901,46 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } } + + test("bucket coalescing is not satisfied") { + def run(testSpec1: BucketedTableTestSpec, testSpec2: BucketedTableTestSpec): Unit = { + Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => + testBucketing( + bucketedTableTestSpecLeft = specs._1, + bucketedTableTestSpecRight = specs._2, + joinCondition = joinCondition(Seq("i", "j"))) + } + } + + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { + // Coalescing buckets is disabled by a config. + run( + BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) + } + + withSQLConf( + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { + // Coalescing buckets is not applied because the difference in the number of buckets (4) + // is greater than max allowed (2). + run( + BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) + } + + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run( + // Coalescing buckets is not applied because the bigger number of buckets (8) is not + // divisible by the smaller number of buckets (7). + BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + BucketedTableTestSpec( + Some(BucketSpec(7, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) + } + } } From 26cfb6374a77f94f83c434428484acb85344b314 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 24 Apr 2020 16:25:59 -0700 Subject: [PATCH 08/21] testing build failure on remote machine --- .../spark/sql/sources/BucketedReadSuite.scala | 50 +++++++++---------- 1 file changed, 24 insertions(+), 26 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 0b67537b37326..1228eca84bdc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -876,32 +876,6 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } - test("bucket coalescing eliminates shuffle") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. - // Currently, sort will be introduced for the side that is coalesced. - val testSpec1 = BucketedTableTestSpec( - Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), - numPartitions = 1, - expectedShuffle = false, - expectedSort = true, - expectedNumOutputPartitions = Some(4)) - val testSpec2 = BucketedTableTestSpec( - Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), - numPartitions = 1, - expectedShuffle = false, - expectedSort = false, - expectedNumOutputPartitions = Some(4)) - - Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => - testBucketing( - bucketedTableTestSpecLeft = specs._1, - bucketedTableTestSpecRight = specs._2, - joinCondition = joinCondition(Seq("i", "j"))) - } - } - } - test("bucket coalescing is not satisfied") { def run(testSpec1: BucketedTableTestSpec, testSpec2: BucketedTableTestSpec): Unit = { Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => @@ -943,4 +917,28 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { Some(BucketSpec(7, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } } + + test("bucket coalescing should eliminate shuffle") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. + // Currently, sort will be introduced for the side that is coalesced. + val testSpec1 = BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = true, + expectedNumOutputPartitions = Some(2)) + val testSpec2 = BucketedTableTestSpec( + Some(BucketSpec(2, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = false, + expectedNumOutputPartitions = Some(2)) + + testBucketing( + bucketedTableTestSpecLeft = testSpec1, + bucketedTableTestSpecRight = testSpec2, + joinCondition = joinCondition(Seq("i", "j"))) + } + } } From ebe062fa01ca393999a0b4ef5906ce998724b993 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 25 Apr 2020 13:16:02 -0700 Subject: [PATCH 09/21] fix incorrect use of SqlConf.get --- .../bucketing/CoalesceBucketsInJoin.scala | 15 ++++-- .../spark/sql/sources/BucketedReadSuite.scala | 50 ++++++++++--------- 2 files changed, 36 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala index eb6d46c9468f1..4de6aa6899e1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala @@ -29,8 +29,6 @@ import org.apache.spark.sql.internal.SQLConf * when the two bucketed tables are inner-joined and they differ in the number of buckets. */ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { - private val sqlConf = SQLConf.get - private def isPlanEligible(plan: LogicalPlan): Boolean = { def forall(plan: LogicalPlan)(p: LogicalPlan => Boolean): Boolean = { p(plan) && plan.children.forall(forall(_)(p)) @@ -53,13 +51,16 @@ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { } } - private def mayCoalesce(numBuckets1: Int, numBuckets2: Int): Option[Int] = { + private def mayCoalesce( + numBuckets1: Int, + numBuckets2: Int, + maxNumBucketsDiff: Int): Option[Int] = { assert(numBuckets1 != numBuckets2) val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller // number of buckets because bucket id is calculated by modding the total number of buckets. if ((large % small == 0) && - ((large - small) <= sqlConf.coalesceBucketsInJoinMaxNumBucketsDiff)) { + ((large - small) <= maxNumBucketsDiff)) { Some(small) } else { None @@ -90,11 +91,15 @@ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { } def apply(plan: LogicalPlan): LogicalPlan = { + val sqlConf = SQLConf.get if (sqlConf.coalesceBucketsInJoinEnabled) { plan transform { case ExtractJoinWithBuckets(join, numLeftBuckets, numRightBuckets) if numLeftBuckets != numRightBuckets => - mayCoalesce(numLeftBuckets, numRightBuckets).map { numCoalescedBuckets => + mayCoalesce( + numLeftBuckets, + numRightBuckets, + sqlConf.coalesceBucketsInJoinMaxNumBucketsDiff).map { numCoalescedBuckets => if (numCoalescedBuckets != numLeftBuckets) { join.copy(left = addCoalesceBuckets(join.left, numCoalescedBuckets)) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 1228eca84bdc6..0b67537b37326 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -876,6 +876,32 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } + test("bucket coalescing eliminates shuffle") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. + // Currently, sort will be introduced for the side that is coalesced. + val testSpec1 = BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = true, + expectedNumOutputPartitions = Some(4)) + val testSpec2 = BucketedTableTestSpec( + Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = false, + expectedNumOutputPartitions = Some(4)) + + Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => + testBucketing( + bucketedTableTestSpecLeft = specs._1, + bucketedTableTestSpecRight = specs._2, + joinCondition = joinCondition(Seq("i", "j"))) + } + } + } + test("bucket coalescing is not satisfied") { def run(testSpec1: BucketedTableTestSpec, testSpec2: BucketedTableTestSpec): Unit = { Seq((testSpec1, testSpec2), (testSpec2, testSpec1)).foreach { specs => @@ -917,28 +943,4 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { Some(BucketSpec(7, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } } - - test("bucket coalescing should eliminate shuffle") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. - // Currently, sort will be introduced for the side that is coalesced. - val testSpec1 = BucketedTableTestSpec( - Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), - numPartitions = 1, - expectedShuffle = false, - expectedSort = true, - expectedNumOutputPartitions = Some(2)) - val testSpec2 = BucketedTableTestSpec( - Some(BucketSpec(2, Seq("i", "j"), Seq("i", "j"))), - numPartitions = 1, - expectedShuffle = false, - expectedSort = false, - expectedNumOutputPartitions = Some(2)) - - testBucketing( - bucketedTableTestSpecLeft = testSpec1, - bucketedTableTestSpecRight = testSpec2, - joinCondition = joinCondition(Seq("i", "j"))) - } - } } From 19b8cd80a62c6c05c79e4253c302071c97d712d1 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 28 Apr 2020 20:49:45 -0700 Subject: [PATCH 10/21] address PR comments --- ...eceBuckets.scala => CoalesceBuckets.scala} | 0 .../bucketing/CoalesceBucketsInJoin.scala | 18 ++-- .../bucketing/CoalesceBucketsSuite.scala | 83 +++++++++++++++++++ 3 files changed, 88 insertions(+), 13 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/{CoalseceBuckets.scala => CoalesceBuckets.scala} (100%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalseceBuckets.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBuckets.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalseceBuckets.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBuckets.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala index 4de6aa6899e1d..42b4fe7b2105c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala @@ -18,15 +18,14 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.internal.SQLConf /** - * This rule injects a hint if one side of two bucketed tables can be coalesced - * when the two bucketed tables are inner-joined and they differ in the number of buckets. + * This rule adds a `CoalesceBuckets` logical plan if one side of two bucketed tables can be + * coalesced when the two bucketed tables are joined and they differ in the number of buckets. */ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { private def isPlanEligible(plan: LogicalPlan): Boolean = { @@ -51,16 +50,12 @@ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { } } - private def mayCoalesce( - numBuckets1: Int, - numBuckets2: Int, - maxNumBucketsDiff: Int): Option[Int] = { + private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { assert(numBuckets1 != numBuckets2) val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller // number of buckets because bucket id is calculated by modding the total number of buckets. - if ((large % small == 0) && - ((large - small) <= maxNumBucketsDiff)) { + if ((large % small == 0) && ((large - small) <= conf.coalesceBucketsInJoinMaxNumBucketsDiff)) { Some(small) } else { None @@ -96,10 +91,7 @@ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { plan transform { case ExtractJoinWithBuckets(join, numLeftBuckets, numRightBuckets) if numLeftBuckets != numRightBuckets => - mayCoalesce( - numLeftBuckets, - numRightBuckets, - sqlConf.coalesceBucketsInJoinMaxNumBucketsDiff).map { numCoalescedBuckets => + mayCoalesce(numLeftBuckets, numRightBuckets, sqlConf).map { numCoalescedBuckets => if (numCoalescedBuckets != numLeftBuckets) { join.copy(left = addCoalesceBuckets(join.left, numCoalescedBuckets)) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala new file mode 100644 index 0000000000000..bc5d32765ace5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala @@ -0,0 +1,83 @@ +/* + * 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.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { + private def newRelation(numBuckets: Int): HadoopFsRelation = HadoopFsRelation( + location = new InMemoryFileIndex(spark, Nil, Map.empty, None), + partitionSchema = StructType(Seq(StructField("a", IntegerType))), + dataSchema = StructType(Seq(StructField("a", IntegerType))), + bucketSpec = Some(BucketSpec(numBuckets, Seq("a"), Seq("a"))), + fileFormat = new ParquetFileFormat(), + options = Map.empty)(spark) + + private def run(numBuckets1: Int, numBuckets2: Int, expectCoalescing: Boolean): Unit = { + Seq((numBuckets1, numBuckets2), (numBuckets2, numBuckets1)).foreach { buckets => + val plan = CoalesceBucketsInJoin( + Join( + LogicalRelation(newRelation(buckets._1)), + LogicalRelation(newRelation(buckets._2)), + Inner, + None, + JoinHint.NONE)) + val coalesced = plan.collect { case c: CoalesceBuckets => c } + if (expectCoalescing) { + assert(coalesced.size == 1) + } else { + assert(coalesced.isEmpty) + } + } + } + + test("bucket coalescing - basic") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { + run(numBuckets1 = 4, numBuckets2 = 8, expectCoalescing = enabled) + } + } + } + + test("bucket coalescing shouldn't be applied when the number of buckets are the same") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(numBuckets1 = 8, numBuckets2 = 8, expectCoalescing = false) + } + } + + test("number of bucket is not divisible by other number of bucket") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(numBuckets1 = 8, numBuckets2 = 3, expectCoalescing = false) + } + } + + test("the difference in the number of buckets is greater than max allowed") { + withSQLConf( + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { + run(numBuckets1 = 8, numBuckets2 = 4, expectCoalescing = false) + } + } +} From 148a6fced050bdf138634c589608396ccdd430ff Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 29 Apr 2020 19:46:30 -0700 Subject: [PATCH 11/21] address PR coments --- .../bucketing/CoalesceBucketsInJoin.scala | 7 ++-- .../bucketing/CoalesceBucketsSuite.scala | 38 +++++++++++++------ 2 files changed, 30 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala index 42b4fe7b2105c..8bb8fb582a9d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} @@ -72,9 +73,9 @@ object CoalesceBucketsInJoin extends Rule[LogicalPlan] { object ExtractJoinWithBuckets { def unapply(plan: LogicalPlan): Option[(Join, Int, Int)] = { plan match { - case join: Join => - val leftBucket = getBucketSpec(join.left) - val rightBucket = getBucketSpec(join.right) + case join @ ExtractEquiJoinKeys(_, _, _, _, left, right, _) => + val leftBucket = getBucketSpec(left) + val rightBucket = getBucketSpec(right) if (leftBucket.isDefined && rightBucket.isDefined) { Some(join, leftBucket.get.numBuckets, rightBucket.get.numBuckets) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala index bc5d32765ace5..b742933c5b5f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.EqualTo import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} @@ -35,15 +36,20 @@ class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { fileFormat = new ParquetFileFormat(), options = Map.empty)(spark) - private def run(numBuckets1: Int, numBuckets2: Int, expectCoalescing: Boolean): Unit = { + private def run( + numBuckets1: Int, + numBuckets2: Int, + expectCoalescing: Boolean, + equiJoin: Boolean): Unit = { Seq((numBuckets1, numBuckets2), (numBuckets2, numBuckets1)).foreach { buckets => - val plan = CoalesceBucketsInJoin( - Join( - LogicalRelation(newRelation(buckets._1)), - LogicalRelation(newRelation(buckets._2)), - Inner, - None, - JoinHint.NONE)) + val left = LogicalRelation(newRelation(buckets._1)) + val right = LogicalRelation(newRelation(buckets._2)) + val condition = if (equiJoin) { + Some(EqualTo(left.output(0), right.output(0))) + } else { + None + } + val plan = CoalesceBucketsInJoin(Join(left, right, Inner, condition, JoinHint.NONE)) val coalesced = plan.collect { case c: CoalesceBuckets => c } if (expectCoalescing) { assert(coalesced.size == 1) @@ -56,20 +62,28 @@ class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { test("bucket coalescing - basic") { Seq(true, false).foreach { enabled => withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { - run(numBuckets1 = 4, numBuckets2 = 8, expectCoalescing = enabled) + run(numBuckets1 = 4, numBuckets2 = 8, expectCoalescing = enabled, equiJoin = true) + } + } + } + + test("bucket coalescing should work only for equi-join") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { + run(numBuckets1 = 4, numBuckets2 = 8, expectCoalescing = false, equiJoin = false) } } } test("bucket coalescing shouldn't be applied when the number of buckets are the same") { withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run(numBuckets1 = 8, numBuckets2 = 8, expectCoalescing = false) + run(numBuckets1 = 8, numBuckets2 = 8, expectCoalescing = false, equiJoin = true) } } test("number of bucket is not divisible by other number of bucket") { withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run(numBuckets1 = 8, numBuckets2 = 3, expectCoalescing = false) + run(numBuckets1 = 8, numBuckets2 = 3, expectCoalescing = false, equiJoin = true) } } @@ -77,7 +91,7 @@ class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { withSQLConf( SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { - run(numBuckets1 = 8, numBuckets2 = 4, expectCoalescing = false) + run(numBuckets1 = 8, numBuckets2 = 4, expectCoalescing = false, equiJoin = true) } } } From eeb0ec7385a9253df0e64316ef2bf069cccf9b6f Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 4 May 2020 19:15:14 -0700 Subject: [PATCH 12/21] Address comments --- .../sql/execution/DataSourceScanExec.scala | 2 +- .../spark/sql/execution/SparkOptimizer.scala | 4 +-- .../execution/bucketing/CoalesceBuckets.scala | 34 ------------------- ....scala => CoalesceBucketsInEquiJoin.scala} | 26 +++++++++++--- ...a => CoalesceBucketsInEquiJoinSuite.scala} | 31 ++++++++++++++--- 5 files changed, 52 insertions(+), 45 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBuckets.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/{CoalesceBucketsInJoin.scala => CoalesceBucketsInEquiJoin.scala} (80%) rename sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/{CoalesceBucketsSuite.scala => CoalesceBucketsInEquiJoinSuite.scala} (76%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index f79f26b6d6725..bf656e210c02d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInEquiJoin import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 12878cc7c3ada..f81085c1e638a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInEquiJoin import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown @@ -45,7 +45,7 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Bucketing", Once, CoalesceBucketsInJoin) :+ + Batch("Bucketing", Once, CoalesceBucketsInEquiJoin) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBuckets.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBuckets.scala deleted file mode 100644 index e1eabd792ffc8..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBuckets.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.spark.sql.execution.bucketing - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.UnaryNode -import org.apache.spark.sql.execution.datasources.LogicalRelation - -/** - * Wraps `LogicalRelation` to provide the number of buckets for coalescing. - */ -case class CoalesceBuckets( - numCoalescedBuckets: Int, - child: LogicalRelation) extends UnaryNode { - require(numCoalescedBuckets > 0, - s"Number of coalesced buckets ($numCoalescedBuckets) must be positive.") - - override def output: Seq[Attribute] = child.output -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoin.scala similarity index 80% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoin.scala index 8bb8fb582a9d1..67dbf3de34d2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoin.scala @@ -18,17 +18,35 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.internal.SQLConf /** - * This rule adds a `CoalesceBuckets` logical plan if one side of two bucketed tables can be - * coalesced when the two bucketed tables are joined and they differ in the number of buckets. + * Wraps `LogicalRelation` to provide the number of buckets for coalescing. */ -object CoalesceBucketsInJoin extends Rule[LogicalPlan] { +case class CoalesceBuckets( + numCoalescedBuckets: Int, + child: LogicalRelation) extends UnaryNode { + require(numCoalescedBuckets > 0, + s"Number of coalesced buckets ($numCoalescedBuckets) must be positive.") + + override def output: Seq[Attribute] = child.output +} + +/** + * This rule adds a `CoalesceBuckets` logical plan if the following conditions are met: + * - Two bucketed tables are joined. + * - Join is the equi-join. + * - The larger bucket number is divisible by the smaller bucket number. + * - "spark.sql.bucketing.coalesceBucketsInJoin.enabled" is set to true. + * - The difference in the number of buckets is less than the value set in + * "spark.sql.bucketing.coalesceBucketsInJoin.maxNumBucketsDiff". + */ +object CoalesceBucketsInEquiJoin extends Rule[LogicalPlan] { private def isPlanEligible(plan: LogicalPlan): Boolean = { def forall(plan: LogicalPlan)(p: LogicalPlan => Boolean): Boolean = { p(plan) && plan.children.forall(forall(_)(p)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala similarity index 76% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala index b742933c5b5f6..09eef152bea52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.EqualTo import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint, LocalRelation, Project} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, ScanOperationWithCoalescedBuckets} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { +class CoalesceBucketsInEquiJoinSuite extends SQLTestUtils with SharedSparkSession { private def newRelation(numBuckets: Int): HadoopFsRelation = HadoopFsRelation( location = new InMemoryFileIndex(spark, Nil, Map.empty, None), partitionSchema = StructType(Seq(StructField("a", IntegerType))), @@ -49,7 +49,7 @@ class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { } else { None } - val plan = CoalesceBucketsInJoin(Join(left, right, Inner, condition, JoinHint.NONE)) + val plan = CoalesceBucketsInEquiJoin(Join(left, right, Inner, condition, JoinHint.NONE)) val coalesced = plan.collect { case c: CoalesceBuckets => c } if (expectCoalescing) { assert(coalesced.size == 1) @@ -94,4 +94,27 @@ class CoalesceBucketsSuite extends SQLTestUtils with SharedSparkSession { run(numBuckets1 = 8, numBuckets2 = 4, expectCoalescing = false, equiJoin = true) } } + + test("test ScanOperationWithCoalescedBuckets extractor") { + val relation = LogicalRelation(newRelation(numBuckets = 4)) + val project = Project(Seq(relation.output(0)), relation) + project match { + case ScanOperationWithCoalescedBuckets(projects, filters, _: LogicalRelation, bucketSpec) => + assert(projects.size == 1) + assert(projects(0) == relation.output(0)) + assert(filters.isEmpty) + assert(bucketSpec.isEmpty) + } + + val projectWithCoalescedBuckets = Project( + Seq(relation.output(0)), + CoalesceBuckets(numCoalescedBuckets = 2, relation)) + projectWithCoalescedBuckets match { + case ScanOperationWithCoalescedBuckets(projects, filters, _: LogicalRelation, bucketSpec) => + assert(projects.size == 1) + assert(projects(0) == relation.output(0)) + assert(filters.isEmpty) + assert(bucketSpec.get == 2) + } + } } From c1e0d2644efb1b5a3811593f64103a538e395a16 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 25 May 2020 11:47:04 -0700 Subject: [PATCH 13/21] address PR comments --- .../sql/execution/DataSourceScanExec.scala | 1 - .../spark/sql/execution/QueryExecution.scala | 2 + .../spark/sql/execution/SparkOptimizer.scala | 2 - ...a => CoalesceBucketsInSortMergeJoin.scala} | 72 ++++----- .../datasources/FileSourceStrategy.scala | 26 +--- .../CoalesceBucketsInEquiJoinSuite.scala | 120 --------------- .../CoalesceBucketsInSortMergeJoinSuite.scala | 141 ++++++++++++++++++ 7 files changed, 174 insertions(+), 190 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/{CoalesceBucketsInEquiJoin.scala => CoalesceBucketsInSortMergeJoin.scala} (52%) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 8dfaae22a11c2..be56014a288e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInEquiJoin import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 1df812d1aa809..b2fce2b1d2253 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInSortMergeJoin import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -295,6 +296,7 @@ object QueryExecution { // as the original plan is hidden behind `AdaptiveSparkPlanExec`. adaptiveExecutionRule.toSeq ++ Seq( + CoalesceBucketsInSortMergeJoin(sparkSession.sessionState.conf), PlanDynamicPruningFilters(sparkSession), PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index f81085c1e638a..33b86a2b5340c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInEquiJoin import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown @@ -45,7 +44,6 @@ class SparkOptimizer( Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ - Batch("Bucketing", Once, CoalesceBucketsInEquiJoin) :+ Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates) :+ Batch("Cleanup filters that cannot be pushed down", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala similarity index 52% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoin.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala index 67dbf3de34d2e..ac59bc410b8aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -18,51 +18,37 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.internal.SQLConf /** - * Wraps `LogicalRelation` to provide the number of buckets for coalescing. - */ -case class CoalesceBuckets( - numCoalescedBuckets: Int, - child: LogicalRelation) extends UnaryNode { - require(numCoalescedBuckets > 0, - s"Number of coalesced buckets ($numCoalescedBuckets) must be positive.") - - override def output: Seq[Attribute] = child.output -} - -/** - * This rule adds a `CoalesceBuckets` logical plan if the following conditions are met: + * This rule coalesces one side of the `SortMergeJoin` if the following conditions are met: * - Two bucketed tables are joined. - * - Join is the equi-join. * - The larger bucket number is divisible by the smaller bucket number. * - "spark.sql.bucketing.coalesceBucketsInJoin.enabled" is set to true. * - The difference in the number of buckets is less than the value set in * "spark.sql.bucketing.coalesceBucketsInJoin.maxNumBucketsDiff". */ -object CoalesceBucketsInEquiJoin extends Rule[LogicalPlan] { - private def isPlanEligible(plan: LogicalPlan): Boolean = { - def forall(plan: LogicalPlan)(p: LogicalPlan => Boolean): Boolean = { +case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { + private def isPlanEligible(plan: SparkPlan): Boolean = { + def forall(plan: SparkPlan)(p: SparkPlan => Boolean): Boolean = { p(plan) && plan.children.forall(forall(_)(p)) } forall(plan) { - case _: Filter | _: Project | _: LogicalRelation => true + case _: FilterExec | _: ProjectExec | _: FileSourceScanExec => true case _ => false } } - private def getBucketSpec(plan: LogicalPlan): Option[BucketSpec] = { + private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { if (isPlanEligible(plan)) { plan.collectFirst { - case _ @ LogicalRelation(r: HadoopFsRelation, _, _, _) if r.bucketSpec.nonEmpty => - r.bucketSpec.get + case f: FileSourceScanExec + if f.relation.bucketSpec.nonEmpty && f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get } } else { None @@ -81,21 +67,21 @@ object CoalesceBucketsInEquiJoin extends Rule[LogicalPlan] { } } - private def addCoalesceBuckets(plan: LogicalPlan, numCoalescedBuckets: Int): LogicalPlan = { + private def updateNumCoalescedBuckets(plan: SparkPlan, numCoalescedBuckets: Int): SparkPlan = { plan.transformUp { - case l @ LogicalRelation(_: HadoopFsRelation, _, _, _) => - CoalesceBuckets(numCoalescedBuckets, l) + case f: FileSourceScanExec => + f.copy(optionalNumCoalescedBuckets = Some(numCoalescedBuckets)) } } - object ExtractJoinWithBuckets { - def unapply(plan: LogicalPlan): Option[(Join, Int, Int)] = { + object ExtractSortMergeJoinWithBuckets { + def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { plan match { - case join @ ExtractEquiJoinKeys(_, _, _, _, left, right, _) => - val leftBucket = getBucketSpec(left) - val rightBucket = getBucketSpec(right) + case s: SortMergeJoinExec => + val leftBucket = getBucketSpec(s.left) + val rightBucket = getBucketSpec(s.right) if (leftBucket.isDefined && rightBucket.isDefined) { - Some(join, leftBucket.get.numBuckets, rightBucket.get.numBuckets) + Some(s, leftBucket.get.numBuckets, rightBucket.get.numBuckets) } else { None } @@ -104,20 +90,18 @@ object CoalesceBucketsInEquiJoin extends Rule[LogicalPlan] { } } - def apply(plan: LogicalPlan): LogicalPlan = { - val sqlConf = SQLConf.get - if (sqlConf.coalesceBucketsInJoinEnabled) { + def apply(plan: SparkPlan): SparkPlan = { + if (conf.coalesceBucketsInJoinEnabled) { plan transform { - case ExtractJoinWithBuckets(join, numLeftBuckets, numRightBuckets) - if numLeftBuckets != numRightBuckets => - mayCoalesce(numLeftBuckets, numRightBuckets, sqlConf).map { numCoalescedBuckets => + case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) + if numLeftBuckets != numRightBuckets => + mayCoalesce(numLeftBuckets, numRightBuckets, conf).map { numCoalescedBuckets => if (numCoalescedBuckets != numLeftBuckets) { - join.copy(left = addCoalesceBuckets(join.left, numCoalescedBuckets)) + smj.copy(left = updateNumCoalescedBuckets(smj.left, numCoalescedBuckets)) } else { - join.copy(right = addCoalesceBuckets(join.right, numCoalescedBuckets)) + smj.copy(right = updateNumCoalescedBuckets(smj.right, numCoalescedBuckets)) } - }.getOrElse(join) - + }.getOrElse(smj) case other => other } } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index d50cdd67e5dc2..512024dff0051 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} -import org.apache.spark.sql.execution.bucketing.CoalesceBuckets import org.apache.spark.util.collection.BitSet /** @@ -138,8 +137,8 @@ object FileSourceStrategy extends Strategy with Logging { } def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ScanOperationWithCoalescedBuckets(projects, filters, - l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _), numCoalescedBuckets) => + case ScanOperation(projects, filters, + l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _)) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read @@ -209,7 +208,7 @@ object FileSourceStrategy extends Strategy with Logging { outputSchema, partitionKeyFilters.toSeq, bucketSet, - numCoalescedBuckets, + None, dataFilters, table.map(_.identifier)) @@ -226,22 +225,3 @@ object FileSourceStrategy extends Strategy with Logging { case _ => Nil } } - -/** - * Extractor that handles `CoalesceBuckets` in the child plan extracted from `ScanOperation`. - */ -object ScanOperationWithCoalescedBuckets { - type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan, Option[Int]) - - def unapply(plan: LogicalPlan): Option[ReturnType] = { - plan match { - case ScanOperation(projects, filters, child) => child match { - case c: CoalesceBuckets => - Some(projects, filters, c.child, Some(c.numCoalescedBuckets)) - case _ => - Some(projects, filters, child, None) - } - case _ => None - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala deleted file mode 100644 index 09eef152bea52..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInEquiJoinSuite.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.spark.sql.execution.bucketing - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.EqualTo -import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint, LocalRelation, Project} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, ScanOperationWithCoalescedBuckets} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} - -class CoalesceBucketsInEquiJoinSuite extends SQLTestUtils with SharedSparkSession { - private def newRelation(numBuckets: Int): HadoopFsRelation = HadoopFsRelation( - location = new InMemoryFileIndex(spark, Nil, Map.empty, None), - partitionSchema = StructType(Seq(StructField("a", IntegerType))), - dataSchema = StructType(Seq(StructField("a", IntegerType))), - bucketSpec = Some(BucketSpec(numBuckets, Seq("a"), Seq("a"))), - fileFormat = new ParquetFileFormat(), - options = Map.empty)(spark) - - private def run( - numBuckets1: Int, - numBuckets2: Int, - expectCoalescing: Boolean, - equiJoin: Boolean): Unit = { - Seq((numBuckets1, numBuckets2), (numBuckets2, numBuckets1)).foreach { buckets => - val left = LogicalRelation(newRelation(buckets._1)) - val right = LogicalRelation(newRelation(buckets._2)) - val condition = if (equiJoin) { - Some(EqualTo(left.output(0), right.output(0))) - } else { - None - } - val plan = CoalesceBucketsInEquiJoin(Join(left, right, Inner, condition, JoinHint.NONE)) - val coalesced = plan.collect { case c: CoalesceBuckets => c } - if (expectCoalescing) { - assert(coalesced.size == 1) - } else { - assert(coalesced.isEmpty) - } - } - } - - test("bucket coalescing - basic") { - Seq(true, false).foreach { enabled => - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { - run(numBuckets1 = 4, numBuckets2 = 8, expectCoalescing = enabled, equiJoin = true) - } - } - } - - test("bucket coalescing should work only for equi-join") { - Seq(true, false).foreach { enabled => - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { - run(numBuckets1 = 4, numBuckets2 = 8, expectCoalescing = false, equiJoin = false) - } - } - } - - test("bucket coalescing shouldn't be applied when the number of buckets are the same") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run(numBuckets1 = 8, numBuckets2 = 8, expectCoalescing = false, equiJoin = true) - } - } - - test("number of bucket is not divisible by other number of bucket") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run(numBuckets1 = 8, numBuckets2 = 3, expectCoalescing = false, equiJoin = true) - } - } - - test("the difference in the number of buckets is greater than max allowed") { - withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { - run(numBuckets1 = 8, numBuckets2 = 4, expectCoalescing = false, equiJoin = true) - } - } - - test("test ScanOperationWithCoalescedBuckets extractor") { - val relation = LogicalRelation(newRelation(numBuckets = 4)) - val project = Project(Seq(relation.output(0)), relation) - project match { - case ScanOperationWithCoalescedBuckets(projects, filters, _: LogicalRelation, bucketSpec) => - assert(projects.size == 1) - assert(projects(0) == relation.output(0)) - assert(filters.isEmpty) - assert(bucketSpec.isEmpty) - } - - val projectWithCoalescedBuckets = Project( - Seq(relation.output(0)), - CoalesceBuckets(numCoalescedBuckets = 2, relation)) - projectWithCoalescedBuckets match { - case ScanOperationWithCoalescedBuckets(projects, filters, _: LogicalRelation, bucketSpec) => - assert(projects.size == 1) - assert(projects(0) == relation.output(0)) - assert(filters.isEmpty) - assert(bucketSpec.get == 2) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala new file mode 100644 index 0000000000000..779971f1fb9dd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala @@ -0,0 +1,141 @@ +/* + * 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.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.execution.{BinaryExecNode, FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkSession { + private def newRelation(numBuckets: Int): HadoopFsRelation = HadoopFsRelation( + location = new InMemoryFileIndex(spark, Nil, Map.empty, None), + partitionSchema = StructType(Seq(StructField("a", IntegerType))), + dataSchema = StructType(Seq(StructField("a", IntegerType))), + bucketSpec = Some(BucketSpec(numBuckets, Seq("a"), Seq("a"))), + fileFormat = new ParquetFileFormat(), + options = Map.empty)(spark) + + private def run( + leftNumBuckets: Int, + rightNumBuckets: Int, + isSortMergeJoin: Boolean, + expectedLeftCoalescedNumBuckets: Option[Int], + expectedRightCoalescedNumBuckets: Option[Int]): Unit = { + val leftRelation = newRelation(leftNumBuckets) + val rightRelation = newRelation(rightNumBuckets) + val left = FileSourceScanExec( + leftRelation, Nil, leftRelation.dataSchema, Nil, None, None, Nil, None) + val right = FileSourceScanExec( + rightRelation, Nil, rightRelation.dataSchema, Nil, None, None, Nil, None) + val join = if (isSortMergeJoin) { + SortMergeJoinExec(Nil, Nil, Inner, None, left, right) + } else { + BroadcastHashJoinExec(Nil, Nil, Inner, BuildLeft, None, left, right) + } + + val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) + + def verify(expected: Option[Int], subPlan: SparkPlan): Unit = { + val coalesced = subPlan.collect { + case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.nonEmpty => + f.optionalNumCoalescedBuckets.get + } + if (expected.isDefined) { + assert(coalesced.size == 1 && coalesced(0) == expected.get) + } else { + assert(coalesced.isEmpty) + } + } + + verify(expectedLeftCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].left) + verify(expectedRightCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].right) + } + + test("bucket coalescing - basic") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run( + leftNumBuckets = 4, + rightNumBuckets = 8, + isSortMergeJoin = true, + expectedLeftCoalescedNumBuckets = None, + expectedRightCoalescedNumBuckets = Some(4)) + } + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { + run( + leftNumBuckets = 4, + rightNumBuckets = 8, + isSortMergeJoin = true, + expectedLeftCoalescedNumBuckets = None, + expectedRightCoalescedNumBuckets = None) + } + } + + test("bucket coalescing should work only for sort merge join") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { + run( + leftNumBuckets = 4, + rightNumBuckets = 8, + isSortMergeJoin = false, + expectedLeftCoalescedNumBuckets = None, + expectedRightCoalescedNumBuckets = None) + } + } + } + + test("bucket coalescing shouldn't be applied when the number of buckets are the same") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run( + leftNumBuckets = 8, + rightNumBuckets = 8, + isSortMergeJoin = true, + expectedLeftCoalescedNumBuckets = None, + expectedRightCoalescedNumBuckets = None) + } + } + + test("number of bucket is not divisible by other number of bucket") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run( + leftNumBuckets = 8, + rightNumBuckets = 3, + isSortMergeJoin = true, + expectedLeftCoalescedNumBuckets = None, + expectedRightCoalescedNumBuckets = None) + } + } + + test("the difference in the number of buckets is greater than max allowed") { + withSQLConf( + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { + run( + leftNumBuckets = 8, + rightNumBuckets = 4, + isSortMergeJoin = true, + expectedLeftCoalescedNumBuckets = None, + expectedRightCoalescedNumBuckets = None) + } + } +} From e54d34bc0cb1013972dcbe38587afce43957ea32 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 30 May 2020 10:53:31 -0700 Subject: [PATCH 14/21] address PR comments --- .../apache/spark/sql/internal/SQLConf.scala | 21 +-- .../CoalesceBucketsInSortMergeJoin.scala | 55 ++++---- .../CoalesceBucketsInSortMergeJoinSuite.scala | 121 +++++++----------- .../spark/sql/sources/BucketedReadSuite.scala | 10 +- 4 files changed, 92 insertions(+), 115 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b77f1d12365de..181df56203082 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2586,21 +2586,21 @@ object SQLConf { .checkValue(_ > 0, "The timeout value must be positive") .createWithDefault(10L) - val COALESCE_BUCKETS_IN_JOIN_ENABLED = - buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled") + val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = + buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled") .doc("When true, if two bucketed tables with the different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + - "of buckets as the other side. This bucket coalescing can happen only when the bigger " + - "number of buckets is divisible by the smaller number of buckets.") + "of buckets as the other side. Bucket coalescing is applied only to sort-merge joins " + + "and only when the bigger number of buckets is divisible by the smaller number of buckets.") .version("3.1.0") .booleanConf .createWithDefault(false) - val COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF = - buildConf("spark.sql.bucketing.coalesceBucketsInJoin.maxNumBucketsDiff") + val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF = + buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxNumBucketsDiff") .doc("The difference in count of two buckets being coalesced should be less than or " + "equal to this value for bucket coalescing to be applied. This configuration only " + - s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set to true.") + s"has an effect when '${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.") .version("3.1.0") .intConf .checkValue(_ > 0, "The difference must be positive.") @@ -2905,10 +2905,11 @@ class SQLConf extends Serializable with Logging { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } - def coalesceBucketsInJoinEnabled: Boolean = getConf(COALESCE_BUCKETS_IN_JOIN_ENABLED) + def coalesceBucketsInSortMergeJoinEnabled: Boolean = + getConf(COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED) - def coalesceBucketsInJoinMaxNumBucketsDiff: Int = - getConf(COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF) + def coalesceBucketsInSortMergeJoinMaxNumBucketsDiff: Int = + getConf(COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF) /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala index ac59bc410b8aa..a876e23cfcae6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -27,40 +27,18 @@ import org.apache.spark.sql.internal.SQLConf * This rule coalesces one side of the `SortMergeJoin` if the following conditions are met: * - Two bucketed tables are joined. * - The larger bucket number is divisible by the smaller bucket number. - * - "spark.sql.bucketing.coalesceBucketsInJoin.enabled" is set to true. + * - COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED is set to true. * - The difference in the number of buckets is less than the value set in - * "spark.sql.bucketing.coalesceBucketsInJoin.maxNumBucketsDiff". + * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF. */ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { - private def isPlanEligible(plan: SparkPlan): Boolean = { - def forall(plan: SparkPlan)(p: SparkPlan => Boolean): Boolean = { - p(plan) && plan.children.forall(forall(_)(p)) - } - - forall(plan) { - case _: FilterExec | _: ProjectExec | _: FileSourceScanExec => true - case _ => false - } - } - - private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { - if (isPlanEligible(plan)) { - plan.collectFirst { - case f: FileSourceScanExec - if f.relation.bucketSpec.nonEmpty && f.optionalNumCoalescedBuckets.isEmpty => - f.relation.bucketSpec.get - } - } else { - None - } - } - private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { assert(numBuckets1 != numBuckets2) val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller // number of buckets because bucket id is calculated by modding the total number of buckets. - if ((large % small == 0) && ((large - small) <= conf.coalesceBucketsInJoinMaxNumBucketsDiff)) { + if ((large % small == 0) && + ((large - small) <= conf.coalesceBucketsInSortMergeJoinMaxNumBucketsDiff)) { Some(small) } else { None @@ -75,6 +53,29 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] } object ExtractSortMergeJoinWithBuckets { + private def isScanOperation(plan: SparkPlan): Boolean = { + def forall(plan: SparkPlan)(p: SparkPlan => Boolean): Boolean = { + p(plan) && plan.children.forall(forall(_)(p)) + } + + forall(plan) { + case _: FilterExec | _: ProjectExec | _: FileSourceScanExec => true + case _ => false + } + } + + private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { + if (isScanOperation(plan)) { + plan.collectFirst { + case f: FileSourceScanExec + if f.relation.bucketSpec.nonEmpty && f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get + } + } else { + None + } + } + def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { plan match { case s: SortMergeJoinExec => @@ -91,7 +92,7 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] } def apply(plan: SparkPlan): SparkPlan = { - if (conf.coalesceBucketsInJoinEnabled) { + if (conf.coalesceBucketsInSortMergeJoinEnabled) { plan transform { case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) if numLeftBuckets != numRightBuckets => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala index 779971f1fb9dd..3a405c6e0ebe6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala @@ -18,16 +18,21 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.optimizer.BuildLeft import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.execution.{BinaryExecNode, FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkSession { + case class BucketSetting( + numBuckets: Int, + expectedCoalescedNumBuckets: Option[Int]) + private def newRelation(numBuckets: Int): HadoopFsRelation = HadoopFsRelation( location = new InMemoryFileIndex(spark, Nil, Map.empty, None), partitionSchema = StructType(Seq(StructField("a", IntegerType))), @@ -37,105 +42,75 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS options = Map.empty)(spark) private def run( - leftNumBuckets: Int, - rightNumBuckets: Int, - isSortMergeJoin: Boolean, - expectedLeftCoalescedNumBuckets: Option[Int], - expectedRightCoalescedNumBuckets: Option[Int]): Unit = { - val leftRelation = newRelation(leftNumBuckets) - val rightRelation = newRelation(rightNumBuckets) - val left = FileSourceScanExec( - leftRelation, Nil, leftRelation.dataSchema, Nil, None, None, Nil, None) - val right = FileSourceScanExec( - rightRelation, Nil, rightRelation.dataSchema, Nil, None, None, Nil, None) - val join = if (isSortMergeJoin) { - SortMergeJoinExec(Nil, Nil, Inner, None, left, right) - } else { - BroadcastHashJoinExec(Nil, Nil, Inner, BuildLeft, None, left, right) - } + bucket1: BucketSetting, + bucket2: BucketSetting, + isSortMergeJoin: Boolean): Unit = { + Seq((bucket1, bucket2), (bucket2, bucket1)).foreach { case (l, r) => + val lRelation = newRelation(l.numBuckets) + val rRelation = newRelation(r.numBuckets) + val lScan = FileSourceScanExec( + lRelation, Nil, lRelation.dataSchema, Nil, None, None, Nil, None) + val rScan = FileSourceScanExec( + rRelation, Nil, rRelation.dataSchema, Nil, None, None, Nil, None) + val join = if (isSortMergeJoin) { + SortMergeJoinExec(Nil, Nil, Inner, None, lScan, rScan) + } else { + BroadcastHashJoinExec(Nil, Nil, Inner, BuildLeft, None, lScan, rScan) + } - val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) + val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) - def verify(expected: Option[Int], subPlan: SparkPlan): Unit = { - val coalesced = subPlan.collect { - case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.nonEmpty => - f.optionalNumCoalescedBuckets.get + def verify(expected: Option[Int], subPlan: SparkPlan): Unit = { + val coalesced = subPlan.collect { + case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.nonEmpty => + f.optionalNumCoalescedBuckets.get + } + if (expected.isDefined) { + assert(coalesced.size == 1 && coalesced(0) == expected.get) + } else { + assert(coalesced.isEmpty) + } } - if (expected.isDefined) { - assert(coalesced.size == 1 && coalesced(0) == expected.get) - } else { - assert(coalesced.isEmpty) - } - } - verify(expectedLeftCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].left) - verify(expectedRightCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].right) + verify(l.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].left) + verify(r.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].right) + } } test("bucket coalescing - basic") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run( - leftNumBuckets = 4, - rightNumBuckets = 8, - isSortMergeJoin = true, - expectedLeftCoalescedNumBuckets = None, - expectedRightCoalescedNumBuckets = Some(4)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run(BucketSetting(4, None), BucketSetting(8, Some(4)), isSortMergeJoin = true) } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { - run( - leftNumBuckets = 4, - rightNumBuckets = 8, - isSortMergeJoin = true, - expectedLeftCoalescedNumBuckets = None, - expectedRightCoalescedNumBuckets = None) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { + run(BucketSetting(4, None), BucketSetting(8, None), isSortMergeJoin = true) } } test("bucket coalescing should work only for sort merge join") { Seq(true, false).foreach { enabled => - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { - run( - leftNumBuckets = 4, - rightNumBuckets = 8, - isSortMergeJoin = false, - expectedLeftCoalescedNumBuckets = None, - expectedRightCoalescedNumBuckets = None) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> enabled.toString) { + run(BucketSetting(4, None), BucketSetting(8, None), isSortMergeJoin = false) } } } test("bucket coalescing shouldn't be applied when the number of buckets are the same") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run( - leftNumBuckets = 8, - rightNumBuckets = 8, - isSortMergeJoin = true, - expectedLeftCoalescedNumBuckets = None, - expectedRightCoalescedNumBuckets = None) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run(BucketSetting(8, None), BucketSetting(8, None), isSortMergeJoin = true) } } test("number of bucket is not divisible by other number of bucket") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { - run( - leftNumBuckets = 8, - rightNumBuckets = 3, - isSortMergeJoin = true, - expectedLeftCoalescedNumBuckets = None, - expectedRightCoalescedNumBuckets = None) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + run(BucketSetting(3, None), BucketSetting(8, None), isSortMergeJoin = true) } } test("the difference in the number of buckets is greater than max allowed") { withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { - run( - leftNumBuckets = 8, - rightNumBuckets = 4, - isSortMergeJoin = true, - expectedLeftCoalescedNumBuckets = None, - expectedRightCoalescedNumBuckets = None) + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { + run(BucketSetting(4, None), BucketSetting(8, None), isSortMergeJoin = true) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 9e29c79206a56..f650da8078e29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -876,7 +876,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } test("bucket coalescing eliminates shuffle") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. // Currently, sort will be introduced for the side that is coalesced. val testSpec1 = BucketedTableTestSpec( @@ -911,7 +911,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { // Coalescing buckets is disabled by a config. run( BucketedTableTestSpec( @@ -921,8 +921,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { // Coalescing buckets is not applied because the difference in the number of buckets (4) // is greater than max allowed (2). run( @@ -932,7 +932,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { run( // Coalescing buckets is not applied because the bigger number of buckets (8) is not // divisible by the smaller number of buckets (7). From eee488479df0035447e228c587d1f2d48c21bde9 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 30 May 2020 10:58:40 -0700 Subject: [PATCH 15/21] fix spacing --- .../execution/bucketing/CoalesceBucketsInSortMergeJoin.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala index a876e23cfcae6..3d48773c2cc2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf * - The difference in the number of buckets is less than the value set in * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF. */ -case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { +case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { assert(numBuckets1 != numBuckets2) val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) From ed2d3162add15eb0df7b28f9e9c1022fa283b2e9 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sun, 31 May 2020 12:02:34 -0700 Subject: [PATCH 16/21] address PR comments --- .../CoalesceBucketsInSortMergeJoin.scala | 82 ++++++++++--------- 1 file changed, 43 insertions(+), 39 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala index 3d48773c2cc2b..48c75f425b494 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -52,45 +52,6 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] } } - object ExtractSortMergeJoinWithBuckets { - private def isScanOperation(plan: SparkPlan): Boolean = { - def forall(plan: SparkPlan)(p: SparkPlan => Boolean): Boolean = { - p(plan) && plan.children.forall(forall(_)(p)) - } - - forall(plan) { - case _: FilterExec | _: ProjectExec | _: FileSourceScanExec => true - case _ => false - } - } - - private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { - if (isScanOperation(plan)) { - plan.collectFirst { - case f: FileSourceScanExec - if f.relation.bucketSpec.nonEmpty && f.optionalNumCoalescedBuckets.isEmpty => - f.relation.bucketSpec.get - } - } else { - None - } - } - - def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { - plan match { - case s: SortMergeJoinExec => - val leftBucket = getBucketSpec(s.left) - val rightBucket = getBucketSpec(s.right) - if (leftBucket.isDefined && rightBucket.isDefined) { - Some(s, leftBucket.get.numBuckets, rightBucket.get.numBuckets) - } else { - None - } - case _ => None - } - } - } - def apply(plan: SparkPlan): SparkPlan = { if (conf.coalesceBucketsInSortMergeJoinEnabled) { plan transform { @@ -110,3 +71,46 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] } } } + +/** + * An extractor that extracts `SortMergeJoinExec` where both sides of the join have the bucketed + * tables and are consisted of only the scan operation. + */ +object ExtractSortMergeJoinWithBuckets { + private def isScanOperation(plan: SparkPlan): Boolean = { + def forall(plan: SparkPlan)(p: SparkPlan => Boolean): Boolean = { + p(plan) && plan.children.forall(forall(_)(p)) + } + + forall(plan) { + case _: FilterExec | _: ProjectExec | _: FileSourceScanExec => true + case _ => false + } + } + + private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { + if (isScanOperation(plan)) { + plan.collectFirst { + case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && + f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get + } + } else { + None + } + } + + def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { + plan match { + case s: SortMergeJoinExec => + val leftBucket = getBucketSpec(s.left) + val rightBucket = getBucketSpec(s.right) + if (leftBucket.isDefined && rightBucket.isDefined) { + Some(s, leftBucket.get.numBuckets, rightBucket.get.numBuckets) + } else { + None + } + case _ => None + } + } +} From 1df3b9fec816862ad5e146d167c3b8de020f3f13 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 10 Jun 2020 11:46:51 -0700 Subject: [PATCH 17/21] Address PR comments --- .../apache/spark/sql/internal/SQLConf.scala | 14 ++--- .../CoalesceBucketsInSortMergeJoin.scala | 52 +++++++++---------- .../CoalesceBucketsInSortMergeJoinSuite.scala | 6 +-- .../spark/sql/sources/BucketedReadSuite.scala | 6 +-- 4 files changed, 34 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 59b25a25e629a..e1b7f3f8df4bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2605,15 +2605,15 @@ object SQLConf { .booleanConf .createWithDefault(false) - val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF = - buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxNumBucketsDiff") - .doc("The difference in count of two buckets being coalesced should be less than or " + + val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO = + buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio") + .doc("The ratio of the number of two buckets being coalesced should be less than or " + "equal to this value for bucket coalescing to be applied. This configuration only " + s"has an effect when '${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.") .version("3.1.0") .intConf .checkValue(_ > 0, "The difference must be positive.") - .createWithDefault(256) + .createWithDefault(10) /** * Holds information about keys that have been deprecated. @@ -2919,12 +2919,6 @@ class SQLConf extends Serializable with Logging { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } - def coalesceBucketsInSortMergeJoinEnabled: Boolean = - getConf(COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED) - - def coalesceBucketsInSortMergeJoinMaxNumBucketsDiff: Int = - getConf(COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF) - /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala index 48c75f425b494..0ceae809392b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -28,8 +28,8 @@ import org.apache.spark.sql.internal.SQLConf * - Two bucketed tables are joined. * - The larger bucket number is divisible by the smaller bucket number. * - COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED is set to true. - * - The difference in the number of buckets is less than the value set in - * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF. + * - The ratio of the number of buckets is less than the value set in + * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO. */ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { @@ -37,8 +37,8 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller // number of buckets because bucket id is calculated by modding the total number of buckets. - if ((large % small == 0) && - ((large - small) <= conf.coalesceBucketsInSortMergeJoinMaxNumBucketsDiff)) { + if (large % small == 0 && + large / small <= conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO)) { Some(small) } else { None @@ -53,21 +53,21 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] } def apply(plan: SparkPlan): SparkPlan = { - if (conf.coalesceBucketsInSortMergeJoinEnabled) { - plan transform { - case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) - if numLeftBuckets != numRightBuckets => - mayCoalesce(numLeftBuckets, numRightBuckets, conf).map { numCoalescedBuckets => - if (numCoalescedBuckets != numLeftBuckets) { - smj.copy(left = updateNumCoalescedBuckets(smj.left, numCoalescedBuckets)) - } else { - smj.copy(right = updateNumCoalescedBuckets(smj.right, numCoalescedBuckets)) - } - }.getOrElse(smj) - case other => other - } - } else { - plan + if (!conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED)) { + return plan + } + + plan transform { + case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) + if numLeftBuckets != numRightBuckets => + mayCoalesce(numLeftBuckets, numRightBuckets, conf).map { numCoalescedBuckets => + if (numCoalescedBuckets != numLeftBuckets) { + smj.copy(left = updateNumCoalescedBuckets(smj.left, numCoalescedBuckets)) + } else { + smj.copy(right = updateNumCoalescedBuckets(smj.right, numCoalescedBuckets)) + } + }.getOrElse(smj) + case other => other } } } @@ -77,15 +77,11 @@ case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] * tables and are consisted of only the scan operation. */ object ExtractSortMergeJoinWithBuckets { - private def isScanOperation(plan: SparkPlan): Boolean = { - def forall(plan: SparkPlan)(p: SparkPlan => Boolean): Boolean = { - p(plan) && plan.children.forall(forall(_)(p)) - } - - forall(plan) { - case _: FilterExec | _: ProjectExec | _: FileSourceScanExec => true - case _ => false - } + private def isScanOperation(plan: SparkPlan): Boolean = plan match { + case f: FilterExec => isScanOperation(f.child) + case p: ProjectExec => isScanOperation(p.child) + case _: FileSourceScanExec => true + case _ => false } private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala index 3a405c6e0ebe6..fdf8f3adb8dd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala @@ -106,11 +106,11 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS } } - test("the difference in the number of buckets is greater than max allowed") { + test("the ratio of the number of buckets is greater than max allowed") { withSQLConf( SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { - run(BucketSetting(4, None), BucketSetting(8, None), isSortMergeJoin = true) + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { + run(BucketSetting(4, None), BucketSetting(16, None), isSortMergeJoin = true) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index f650da8078e29..2508187e2ae4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -922,12 +922,12 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { withSQLConf( SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_NUM_BUCKETS_DIFF.key -> "2") { - // Coalescing buckets is not applied because the difference in the number of buckets (4) + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { + // Coalescing buckets is not applied because the ratio of the number of buckets (3) // is greater than max allowed (2). run( BucketedTableTestSpec( - Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), + Some(BucketSpec(12, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = false), BucketedTableTestSpec( Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } From cf5b83535c0998a91a3ad2e232516a4a219fff92 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 11 Jun 2020 20:43:27 -0700 Subject: [PATCH 18/21] Address PR comments --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e1b7f3f8df4bc..8e76b54a890e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2613,7 +2613,7 @@ object SQLConf { .version("3.1.0") .intConf .checkValue(_ > 0, "The difference must be positive.") - .createWithDefault(10) + .createWithDefault(4) /** * Holds information about keys that have been deprecated. From e23126816cf9e9d538479c389c32b87708d246e8 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 17 Jun 2020 20:06:26 -0700 Subject: [PATCH 19/21] address PR comments --- .../apache/spark/sql/internal/SQLConf.scala | 5 +- .../sql/execution/DataSourceScanExec.scala | 3 +- .../CoalesceBucketsInSortMergeJoin.scala | 38 +++-- .../CoalesceBucketsInSortMergeJoinSuite.scala | 134 +++++++++++++----- .../spark/sql/sources/BucketedReadSuite.scala | 40 ++++++ 5 files changed, 176 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index cf532d0571a64..a82c4b5a153b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2617,12 +2617,15 @@ object SQLConf { .booleanConf .createWithDefault(false) - val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = + val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled") .doc("When true, if two bucketed tables with the different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + "of buckets as the other side. Bucket coalescing is applied only to sort-merge joins " + "and only when the bigger number of buckets is divisible by the smaller number of buckets.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO = buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index be56014a288e2..71ade845a67f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -360,7 +360,8 @@ case class FileSourceScanExec( spec.numBuckets } metadata + ("SelectedBucketsCount" -> - s"$numSelectedBuckets out of ${spec.numBuckets}") + (s"$numSelectedBuckets out of ${spec.numBuckets}" + + optionalNumCoalescedBuckets.map { b => s" (Coalesced to $b)"}.getOrElse(""))) } getOrElse { metadata } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala index 0ceae809392b4..3bb0597ecd87c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -26,6 +28,7 @@ import org.apache.spark.sql.internal.SQLConf /** * This rule coalesces one side of the `SortMergeJoin` if the following conditions are met: * - Two bucketed tables are joined. + * - Join keys match with output partition expressions on their respective sides. * - The larger bucket number is divisible by the smaller bucket number. * - COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED is set to true. * - The ratio of the number of buckets is less than the value set in @@ -85,20 +88,37 @@ object ExtractSortMergeJoinWithBuckets { } private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { - if (isScanOperation(plan)) { - plan.collectFirst { - case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && - f.optionalNumCoalescedBuckets.isEmpty => - f.relation.bucketSpec.get - } - } else { - None + plan.collectFirst { + case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && + f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get } } + /** + * The join keys should match with expressions for output partitioning. Note that + * the ordering does not matter because it will be handled in `EnsureRequirements`. + */ + private def satisfiesOutputPartitioning( + keys: Seq[Expression], + partitioning: Partitioning): Boolean = { + partitioning match { + case HashPartitioning(exprs, _) if exprs.length == keys.length => + exprs.forall(e => keys.exists(_.semanticEquals(e))) + case _ => false + } + } + + private def isApplicable(s: SortMergeJoinExec): Boolean = { + isScanOperation(s.left) && + isScanOperation(s.right) && + satisfiesOutputPartitioning(s.leftKeys, s.left.outputPartitioning) && + satisfiesOutputPartitioning(s.rightKeys, s.right.outputPartitioning) + } + def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { plan match { - case s: SortMergeJoinExec => + case s: SortMergeJoinExec if isApplicable(s) => val leftBucket = getBucketSpec(s.left) val rightBucket = getBucketSpec(s.right) if (leftBucket.isDefined && rightBucket.isDefined) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala index fdf8f3adb8dd4..f4b805a2e44de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala @@ -18,44 +18,69 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.optimizer.BuildLeft import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.execution.{BinaryExecNode, FileSourceScanExec, SparkPlan} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, PartitionSpec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StructType} class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkSession { - case class BucketSetting( + case class RelationSetting( + cols: Seq[Attribute], numBuckets: Int, expectedCoalescedNumBuckets: Option[Int]) - private def newRelation(numBuckets: Int): HadoopFsRelation = HadoopFsRelation( - location = new InMemoryFileIndex(spark, Nil, Map.empty, None), - partitionSchema = StructType(Seq(StructField("a", IntegerType))), - dataSchema = StructType(Seq(StructField("a", IntegerType))), - bucketSpec = Some(BucketSpec(numBuckets, Seq("a"), Seq("a"))), - fileFormat = new ParquetFileFormat(), - options = Map.empty)(spark) - - private def run( - bucket1: BucketSetting, - bucket2: BucketSetting, - isSortMergeJoin: Boolean): Unit = { - Seq((bucket1, bucket2), (bucket2, bucket1)).foreach { case (l, r) => - val lRelation = newRelation(l.numBuckets) - val rRelation = newRelation(r.numBuckets) - val lScan = FileSourceScanExec( - lRelation, Nil, lRelation.dataSchema, Nil, None, None, Nil, None) - val rScan = FileSourceScanExec( - rRelation, Nil, rRelation.dataSchema, Nil, None, None, Nil, None) - val join = if (isSortMergeJoin) { - SortMergeJoinExec(Nil, Nil, Inner, None, lScan, rScan) + object RelationSetting { + def apply(numBuckets: Int, expectedCoalescedNumBuckets: Option[Int]): RelationSetting = { + val cols = Seq(AttributeReference("i", IntegerType)()) + RelationSetting(cols, numBuckets, expectedCoalescedNumBuckets) + } + } + + case class JoinSetting( + leftKeys: Seq[Attribute], + rightKeys: Seq[Attribute], + leftRelation: RelationSetting, + rightRelation: RelationSetting, + isSortMergeJoin: Boolean) + + object JoinSetting { + def apply(l: RelationSetting, r: RelationSetting, isSortMergeJoin: Boolean): JoinSetting = { + JoinSetting(l.cols, r.cols, l, r, isSortMergeJoin) + } + } + + private def newFileSourceScanExec(setting: RelationSetting): FileSourceScanExec = { + val relation = HadoopFsRelation( + location = new InMemoryFileIndex(spark, Nil, Map.empty, None), + partitionSchema = PartitionSpec.emptySpec.partitionColumns, + dataSchema = StructType.fromAttributes(setting.cols), + bucketSpec = Some(BucketSpec(setting.numBuckets, setting.cols.map(_.name), Nil)), + fileFormat = new ParquetFileFormat(), + options = Map.empty)(spark) + FileSourceScanExec(relation, setting.cols, relation.dataSchema, Nil, None, None, Nil, None) + } + + private def run(setting: JoinSetting): Unit = { + val swappedSetting = setting.copy( + leftKeys = setting.rightKeys, + rightKeys = setting.leftKeys, + leftRelation = setting.rightRelation, + rightRelation = setting.leftRelation) + + Seq(setting, swappedSetting).foreach { case s => + val lScan = newFileSourceScanExec(s.leftRelation) + val rScan = newFileSourceScanExec(s.rightRelation) + val join = if (s.isSortMergeJoin) { + SortMergeJoinExec(s.leftKeys, s.rightKeys, Inner, None, lScan, rScan) } else { - BroadcastHashJoinExec(Nil, Nil, Inner, BuildLeft, None, lScan, rScan) + BroadcastHashJoinExec( + s.leftKeys, s.rightKeys, Inner, BuildLeft, None, lScan, rScan) } val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) @@ -72,37 +97,39 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS } } - verify(l.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].left) - verify(r.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].right) + verify(s.leftRelation.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].left) + verify(s.rightRelation.expectedCoalescedNumBuckets, plan.asInstanceOf[BinaryExecNode].right) } } test("bucket coalescing - basic") { withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(BucketSetting(4, None), BucketSetting(8, Some(4)), isSortMergeJoin = true) + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, Some(4)), isSortMergeJoin = true)) } withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { - run(BucketSetting(4, None), BucketSetting(8, None), isSortMergeJoin = true) + run(JoinSetting(RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = true)) } } test("bucket coalescing should work only for sort merge join") { Seq(true, false).foreach { enabled => withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> enabled.toString) { - run(BucketSetting(4, None), BucketSetting(8, None), isSortMergeJoin = false) + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = false)) } } } test("bucket coalescing shouldn't be applied when the number of buckets are the same") { withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(BucketSetting(8, None), BucketSetting(8, None), isSortMergeJoin = true) + run(JoinSetting(RelationSetting(8, None), RelationSetting(8, None), isSortMergeJoin = true)) } } test("number of bucket is not divisible by other number of bucket") { withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(BucketSetting(3, None), BucketSetting(8, None), isSortMergeJoin = true) + run(JoinSetting(RelationSetting(3, None), RelationSetting(8, None), isSortMergeJoin = true)) } } @@ -110,7 +137,48 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS withSQLConf( SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { - run(BucketSetting(4, None), BucketSetting(16, None), isSortMergeJoin = true) + run(JoinSetting(RelationSetting(4, None), RelationSetting(16, None), isSortMergeJoin = true)) + } + } + + test("join keys should match with output partitioning") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + val lCols = Seq( + AttributeReference("l1", IntegerType)(), + AttributeReference("l2", IntegerType)()) + val rCols = Seq( + AttributeReference("r1", IntegerType)(), + AttributeReference("r2", IntegerType)()) + + val lRel = RelationSetting(lCols, 4, None) + val rRel = RelationSetting(rCols, 8, None) + + // The following should not be coalesced because join keys do not match with output + // partitioning (missing one expression). + run(JoinSetting( + leftKeys = Seq(lCols.head), + rightKeys = Seq(rCols.head), + leftRelation = lRel, + rightRelation = rRel, + isSortMergeJoin = true)) + + // The following should not be coalesced because join keys do not match with output + // partitioning (more expressions). + run(JoinSetting( + leftKeys = lCols :+ AttributeReference("l3", IntegerType)(), + rightKeys = rCols :+ AttributeReference("r3", IntegerType)(), + leftRelation = lRel, + rightRelation = rRel, + isSortMergeJoin = true)) + + // The following will be coalesced since ordering should not matter because it will be + // adjusted in `EnsureRequirements`. + run(JoinSetting( + leftKeys = lCols.reverse, + rightKeys = rCols.reverse, + leftRelation = lRel, + rightRelation = RelationSetting(rCols, 8, Some(4)), + isSortMergeJoin = true)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 2508187e2ae4c..b6767eb3132ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -942,4 +942,44 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { Some(BucketSpec(7, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } } + + test("bucket coalescing is applied when join expressions match with partitioning expressions") { + withTable("t1", "t2") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") + df2.write.format("parquet").bucketBy(4, "i", "j").saveAsTable("t2") + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + def verify( + query: String, + expectedNumShuffles: Int, + expectedCoalescedNumBuckets: Option[Int]): Unit = { + val plan = sql(query).queryExecution.executedPlan + val shuffles = plan.collect { case s: ShuffleExchangeExec => s } + assert(shuffles.length == expectedNumShuffles) + + val scans = plan.collect { + case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + } + if (expectedCoalescedNumBuckets.isDefined) { + assert(scans.length == 1) + assert(scans(0).optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } else { + assert(scans.isEmpty) + } + } + + // Coalescing applied since join expressions match with the bucket columns. + verify("SELECT * FROM t1 JOIN t2 ON t1.i = t2.i AND t1.j = t2.j", 0, Some(4)) + // Coalescing applied when columns are aliased. + verify( + "SELECT * FROM t1 JOIN (SELECT i AS x, j AS y FROM t2) ON t1.i = x AND t1.j = y", + 0, + Some(4)) + // Coalescing is not applied when join expressions do not match with bucket columns. + verify("SELECT * FROM t1 JOIN t2 ON t1.i = t2.i", 2, None) + } + } + } } From 38c7d6e57fc7adf34b18839115a880ca086eb2b6 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 18 Jun 2020 10:48:03 -0700 Subject: [PATCH 20/21] Address PR comments --- .../spark/sql/execution/DataSourceScanExec.scala | 2 +- .../CoalesceBucketsInSortMergeJoinSuite.scala | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 71ade845a67f4..458e11b97db6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -558,7 +558,7 @@ case class FileSourceScanExec( }.getOrElse(Array.empty) FilePartition(bucketId, partitionedFiles) } - } getOrElse { + }.getOrElse { Seq.tabulate(bucketSpec.numBuckets) { bucketId => FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala index f4b805a2e44de..6a70045c55e64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala @@ -181,4 +181,14 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS isSortMergeJoin = true)) } } + + test("FileSourceScanExec's metadata should be updated with coalesced info") { + val scan = newFileSourceScanExec(RelationSetting(8, None)) + val value = scan.metadata("SelectedBucketsCount") + assert(value === "8 out of 8") + + val scanWithCoalescing = scan.copy(optionalNumCoalescedBuckets = Some(4)) + val valueWithCoalescing = scanWithCoalescing.metadata("SelectedBucketsCount") + assert(valueWithCoalescing == "8 out of 8 (Coalesced to 4)") + } } From 62a04a3e4d94e63b3787533f619e368a7e8d59f6 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 18 Jun 2020 21:51:59 -0700 Subject: [PATCH 21/21] Address PR comment --- .../org/apache/spark/sql/ExplainSuite.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 5aeecd2df91e9..1ad97185a564a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -343,6 +343,23 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite assert(getNormalizedExplain(df1, FormattedMode) === getNormalizedExplain(df2, FormattedMode)) } } + + test("Coalesced bucket info should be a part of explain string") { + withTable("t1", "t2") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + Seq(1, 2).toDF("i").write.bucketBy(8, "i").saveAsTable("t1") + Seq(2, 3).toDF("i").write.bucketBy(4, "i").saveAsTable("t2") + val df1 = spark.table("t1") + val df2 = spark.table("t2") + val joined = df1.join(df2, df1("i") === df2("i")) + checkKeywordsExistsInExplain( + joined, + SimpleMode, + "SelectedBucketsCount: 8 out of 8 (Coalesced to 4)" :: Nil: _*) + } + } + } } class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite {