forked from Kyligence/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KE-42300 support split source partition
- Loading branch information
Yu Gan
committed
Oct 17, 2023
1 parent
82c4992
commit e913aa8
Showing
7 changed files
with
294 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
99 changes: 99 additions & 0 deletions
99
sql/core/src/main/scala/org/apache/spark/sql/execution/split/SplitExec.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,99 @@ | ||
/* | ||
* 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.split | ||
|
||
import scala.annotation.tailrec | ||
import scala.reflect.ClassTag | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.Attribute | ||
import org.apache.spark.sql.execution.{SparkPlan, SQLExecution, UnaryExecNode} | ||
import org.apache.spark.sql.execution.datasources.FileScanRDD | ||
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
|
||
case class SplitExec(maxExpandNum: Int, thresholdSize: Long, child: SparkPlan) | ||
extends UnaryExecNode { | ||
|
||
/** | ||
* @return All metrics containing metrics of this SparkPlan. | ||
*/ | ||
override lazy val metrics: Map[String, SQLMetric] = Map( | ||
"originPartNum" -> SQLMetrics.createMetric(sparkContext, "origin partition num"), | ||
"expandPartNum" -> SQLMetrics.createMetric(sparkContext, "expand partition num")) | ||
|
||
/** | ||
* Returns the name of this type of TreeNode. Defaults to the class name. | ||
* Note that we remove the "Exec" suffix for physical operators here. | ||
*/ | ||
override def nodeName: String = "SplitSourcePartition" | ||
|
||
override def output: Seq[Attribute] = child.output | ||
|
||
override protected def withNewChildInternal(newChild: SparkPlan): SplitExec = | ||
copy(child = newChild) | ||
|
||
/** | ||
* Produces the result of the query as an `RDD[InternalRow]` | ||
* | ||
* Overridden by concrete implementations of SparkPlan. | ||
*/ | ||
override protected def doExecute(): RDD[InternalRow] = { | ||
doSplit(child.execute()) | ||
} | ||
|
||
private def doSplit[U: ClassTag](prev: RDD[U]): RDD[U] = { | ||
val prevPartNum = prev.getNumPartitions | ||
metrics("originPartNum").set(prevPartNum) | ||
// default: do nothing | ||
metrics("expandPartNum").set(prevPartNum) | ||
val sourceSize = evalSourceSize(prev) | ||
val after = sourceSize | ||
.map { size => | ||
if (size < thresholdSize) { | ||
// If source size is tiny, split will not be profitable. | ||
prev | ||
} else { | ||
val expandPartNum = maxExpandNum min sparkContext.defaultParallelism | ||
if (expandPartNum < (prevPartNum << 1)) { | ||
// If expansion scale is tiny, split will also not be profitable. | ||
prev | ||
} else { | ||
metrics("expandPartNum").set(expandPartNum) | ||
// Maybe we could find better ways than `coalesce` to redistribute the partition data. | ||
prev.coalesce(expandPartNum, shuffle = true) | ||
} | ||
} | ||
} | ||
.getOrElse(prev) | ||
|
||
// update metrics | ||
val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) | ||
SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) | ||
after | ||
} | ||
|
||
@tailrec | ||
private def evalSourceSize[U: ClassTag](prev: RDD[U]): Option[Long] = | ||
prev match { | ||
case f: FileScanRDD => Some(f.partitionFilesTotalLength) | ||
case r if r.dependencies.isEmpty => None | ||
case other => evalSourceSize(other.firstParent) | ||
} | ||
|
||
} |
61 changes: 61 additions & 0 deletions
61
sql/core/src/main/scala/org/apache/spark/sql/execution/split/SplitSourcePartition.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
/* | ||
* 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.split | ||
|
||
import scala.annotation.tailrec | ||
|
||
import org.apache.spark.sql.catalyst.rules.Rule | ||
import org.apache.spark.sql.execution._ | ||
import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec | ||
|
||
object SplitSourcePartition extends Rule[SparkPlan] { | ||
|
||
override def apply(plan: SparkPlan): SparkPlan = { | ||
if (!plan.conf.splitSourcePartitionEnabled || | ||
plan.find(_.isInstanceOf[DataSourceScanExec]).isEmpty) { | ||
return plan | ||
} | ||
|
||
val r = plan.transformOnceWithPruning(shouldBreak) { | ||
case p if p != null && p.isInstanceOf[DataSourceScanExec] => | ||
val maxExpandNum = plan.conf.splitSourcePartitionMaxExpandNum | ||
val thresholdSize = plan.conf.splitSourcePartitionThreshold | ||
SplitExec(maxExpandNum, thresholdSize, p) | ||
case other => other | ||
} | ||
r | ||
} | ||
|
||
private def shouldBreak(plan: SparkPlan): Boolean = | ||
plan match { | ||
case BroadcastExchangeExec(_, c) => askChild(c) | ||
/* case p if !p.requiredChildDistribution.forall(_ == UnspecifiedDistribution) => | ||
p.children.exists(supportSplit) */ | ||
case _ => false | ||
} | ||
|
||
@tailrec | ||
private def askChild(plan: SparkPlan): Boolean = | ||
plan match { | ||
case n if n == null => false | ||
case l: LeafExecNode => l.isInstanceOf[DataSourceScanExec] | ||
case u: UnaryExecNode => askChild(u.child) | ||
case _ => false | ||
} | ||
|
||
} |
71 changes: 71 additions & 0 deletions
71
sql/core/src/test/scala/org/apache/spark/sql/execution/SplitSourcePartitionSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* 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 | ||
|
||
import org.apache.spark.sql.{QueryTest, SaveMode} | ||
import org.apache.spark.sql.execution.split.SplitExec | ||
import org.apache.spark.sql.functions.col | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.test.SharedSparkSession | ||
|
||
class SplitSourcePartitionSuite extends QueryTest with SharedSparkSession { | ||
|
||
private val TABLE_FORMAT: String = "parquet" | ||
|
||
test("split source partition") { | ||
withSQLConf( | ||
SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", | ||
SQLConf.FILES_MIN_PARTITION_NUM.key -> "1", | ||
SQLConf.SPLIT_SOURCE_PARTITION_ENABLED.key -> "true", | ||
SQLConf.SPLIT_SOURCE_PARTITION_THRESHOLD.key -> "1B") { | ||
withTable("ssp_t1", "ssp_t2") { | ||
spark | ||
.range(10) | ||
.select(col("id"), col("id").as("k")) | ||
.write | ||
.mode(SaveMode.Overwrite) | ||
.format(TABLE_FORMAT) | ||
.saveAsTable("ssp_t1") | ||
|
||
spark | ||
.range(5) | ||
.select(col("id"), col("id").as("k")) | ||
.write | ||
.mode(SaveMode.Overwrite) | ||
.format(TABLE_FORMAT) | ||
.saveAsTable("ssp_t2") | ||
|
||
val df = sql(""" | ||
|SELECT ssp_t1.id, ssp_t2.k | ||
|FROM ssp_t1 INNER JOIN ssp_t2 ON ssp_t1.k = ssp_t2.k | ||
|WHERE ssp_t2.id < 2 | ||
|""".stripMargin) | ||
|
||
val plan = df.queryExecution.executedPlan | ||
assertResult(1, "SplitExec applied.")(plan.collectWithSubqueries { case e: SplitExec => | ||
e | ||
}.size) | ||
|
||
assertResult(spark.sparkContext.defaultParallelism, "split partitions.")( | ||
df.rdd.partitions.length) | ||
} | ||
|
||
} | ||
} | ||
|
||
} |