Skip to content
This repository was archived by the owner on Jun 14, 2024. It is now read-only.

Commit 8bec7c5

Browse files
committed
Check and remove unnecessary shuffle added by Hybrid Scan
1 parent ec1dfb0 commit 8bec7c5

File tree

4 files changed

+150
-17
lines changed

4 files changed

+150
-17
lines changed

src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala

+6
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,12 @@ object IndexConstants {
4747
"spark.hyperspace.index.hybridscan.maxAppendedRatio"
4848
val INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD_DEFAULT = "0.3"
4949

50+
// If this config is enabled, Hybrid Scan won't be applied when it causes unnecessary shuffle
51+
// to merge appended data.
52+
val INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED =
53+
"spark.hyperspace.index.hybridscan.shuffleCheck.enabled"
54+
val INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED_DEFAULT = "true"
55+
5056
// Identifier injected to HadoopFsRelation as an option if an index is applied.
5157
// Currently, the identifier is added to options field of HadoopFsRelation.
5258
// In Spark 3.0, we could utilize TreeNodeTag to mark the identifier for each plan.

src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala

+45-9
Original file line numberDiff line numberDiff line change
@@ -24,13 +24,17 @@ import org.apache.spark.sql.catalyst.analysis.CleanupAliases
2424
import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, AttributeSet, EqualTo, Expression}
2525
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
2626
import org.apache.spark.sql.catalyst.rules.Rule
27+
import org.apache.spark.sql.execution.SparkPlan
2728
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
29+
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
30+
import org.apache.spark.sql.execution.joins.SortMergeJoinExec
2831

2932
import com.microsoft.hyperspace.{ActiveSparkSession, Hyperspace}
3033
import com.microsoft.hyperspace.actions.Constants
3134
import com.microsoft.hyperspace.index._
3235
import com.microsoft.hyperspace.index.rankers.JoinIndexRanker
3336
import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEventLogging, HyperspaceIndexUsageEvent}
37+
import com.microsoft.hyperspace.util.HyperspaceConf
3438
import com.microsoft.hyperspace.util.ResolverUtils._
3539

3640
/**
@@ -68,18 +72,54 @@ object JoinIndexRule
6872
right =
6973
RuleUtils.transformPlanToUseIndex(spark, rIndex, r, useBucketSpec = true))
7074

75+
def getShuffleCnt(sparkPlan: SparkPlan): Long = {
76+
sparkPlan.collect { case _: ShuffleExchangeExec => true }.length
77+
}
78+
79+
val resultPlan =
80+
if (!HyperspaceConf.hybridScanEnabled(spark) || !HyperspaceConf
81+
.hybridScanShuffleCheckEnabled(spark)) {
82+
updatedPlan
83+
} else {
84+
val shuffleCntPair = spark.sessionState
85+
.executePlan(updatedPlan)
86+
.executedPlan
87+
.collect {
88+
case smj: SortMergeJoinExec =>
89+
(getShuffleCnt(smj.left), getShuffleCnt(smj.right))
90+
}
91+
assert(shuffleCntPair.length <= 1)
92+
shuffleCntPair.headOption.flatMap {
93+
// If the number of shuffle is 2, the candidate index pair cannot remove
94+
// the shuffles in both left and right for join and also Hybrid Scan causes
95+
// an additional shuffle for merging appended files. We don't apply the index
96+
// for the child with 2 shuffle nodes using JoinIndexRule.
97+
// However, the child node is still applicable for FilterIndexRule.
98+
case (leftCnt, _) if leftCnt == 2 => Some(updatedPlan.copy(left = l))
99+
case (_, rightCnt) if rightCnt == 2 => Some(updatedPlan.copy(right = r))
100+
case _ => None
101+
}.getOrElse {
102+
updatedPlan
103+
}
104+
}
105+
106+
val appliedIndexes = resultPlan match {
107+
case j: Join if j.left.equals(l) => Seq(rIndex)
108+
case j: Join if j.right.equals(r) => Seq(lIndex)
109+
case _ => Seq(lIndex, rIndex)
110+
}
111+
71112
logEvent(
72113
HyperspaceIndexUsageEvent(
73114
AppInfo(
74115
sparkContext.sparkUser,
75116
sparkContext.applicationId,
76117
sparkContext.appName),
77-
Seq(lIndex, rIndex),
118+
appliedIndexes,
78119
join.toString,
79-
updatedPlan.toString,
120+
resultPlan.toString,
80121
"Join index rule applied."))
81-
82-
updatedPlan
122+
resultPlan
83123
}
84124
.getOrElse(join)
85125
} catch {
@@ -325,11 +365,7 @@ object JoinIndexRule
325365
compatibleIndexPairs.map(
326366
indexPairs =>
327367
JoinIndexRanker
328-
.rank(
329-
spark,
330-
leftRel,
331-
rightRel,
332-
indexPairs)
368+
.rank(spark, leftRel, rightRel, indexPairs)
333369
.head)
334370
}
335371

src/main/scala/com/microsoft/hyperspace/util/HyperspaceConf.scala

+8
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,14 @@ object HyperspaceConf {
3636
hybridScanDeletedRatioThreshold(spark) > 0.0
3737
}
3838

39+
def hybridScanShuffleCheckEnabled(spark: SparkSession): Boolean = {
40+
spark.conf
41+
.get(
42+
IndexConstants.INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED,
43+
IndexConstants.INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED_DEFAULT)
44+
.toBoolean
45+
}
46+
3947
def optimizeFileSizeThreshold(spark: SparkSession): Long = {
4048
spark.conf
4149
.get(

src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala

+91-8
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,10 @@ import org.apache.hadoop.conf.Configuration
2020
import org.apache.hadoop.fs.Path
2121
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row}
2222
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
23-
import org.apache.spark.sql.execution.SortExec
23+
import org.apache.spark.sql.execution.{SortExec, SparkPlan => SparkPlanNode}
2424
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation}
2525
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
26+
import org.apache.spark.sql.execution.joins.SortMergeJoinExec
2627

2728
import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData, TestConfig, TestUtils}
2829
import com.microsoft.hyperspace.index.IndexConstants.{GLOBBING_PATTERN_KEY, REFRESH_MODE_INCREMENTAL, REFRESH_MODE_QUICK}
@@ -670,13 +671,95 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
670671

671672
// Refreshed index as quick mode can be applied with Hybrid Scan config.
672673
withSQLConf(TestConfig.HybridScanEnabled: _*) {
673-
spark.disableHyperspace()
674-
val dfWithHyperspaceDisabled = query()
675-
val basePlan = dfWithHyperspaceDisabled.queryExecution.optimizedPlan
676-
spark.enableHyperspace()
677-
val dfWithHyperspaceEnabled = query()
678-
assert(!basePlan.equals(dfWithHyperspaceEnabled.queryExecution.optimizedPlan))
679-
checkAnswer(dfWithHyperspaceDisabled, dfWithHyperspaceEnabled)
674+
spark.disableHyperspace()
675+
val dfWithHyperspaceDisabled = query()
676+
val basePlan = dfWithHyperspaceDisabled.queryExecution.optimizedPlan
677+
spark.enableHyperspace()
678+
val dfWithHyperspaceEnabled = query()
679+
assert(!basePlan.equals(dfWithHyperspaceEnabled.queryExecution.optimizedPlan))
680+
checkAnswer(dfWithHyperspaceDisabled, dfWithHyperspaceEnabled)
681+
}
682+
}
683+
}
684+
}
685+
686+
test("Verify Hybrid Scan is not applied when shuffle is not removed.") {
687+
withTempPathAsString { testPath =>
688+
val indexConfig = IndexConfig("indexRight", Seq("c2"), Seq("c4"))
689+
val indexConfig2 = IndexConfig("indexLeft", Seq("c2"), Seq("c3"))
690+
import spark.implicits._
691+
SampleData.testData
692+
.toDF("c1", "c2", "c3", "c4", "c5")
693+
.limit(10)
694+
.write
695+
.parquet(testPath)
696+
val df = spark.read.load(testPath)
697+
698+
withSQLConf(IndexConstants.INDEX_NUM_BUCKETS -> "11") {
699+
// Create an index with bucket num 11.
700+
hyperspace.createIndex(df, indexConfig)
701+
}
702+
withSQLConf(IndexConstants.INDEX_NUM_BUCKETS -> "12") {
703+
// Create an index with bucket num 12.
704+
hyperspace.createIndex(df, indexConfig2)
705+
}
706+
707+
// Append to original data.
708+
SampleData.testData
709+
.toDF("c1", "c2", "c3", "c4", "c5")
710+
.limit(3)
711+
.write
712+
.mode("append")
713+
.parquet(testPath)
714+
715+
{
716+
// Create a join query.
717+
val df2 = spark.read.parquet(testPath)
718+
719+
def query(): DataFrame = {
720+
df2.select("c2", "c3").join(df2.select("c2", "c4"), "c2")
721+
}
722+
723+
val inputFiles = df.inputFiles
724+
val appendedFiles = df2.inputFiles.diff(inputFiles).map(new Path(_))
725+
726+
spark.enableHyperspace()
727+
withSQLConf(TestConfig.HybridScanEnabled: _*) {
728+
def getShuffleCnt(sparkPlan: SparkPlanNode): Long = {
729+
sparkPlan.collect { case _: ShuffleExchangeExec => true }.length
730+
}
731+
withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED -> "false") {
732+
val execPlan = query().queryExecution.executedPlan
733+
val shuffleCnt = execPlan.collect {
734+
case smj: SortMergeJoinExec =>
735+
(getShuffleCnt(smj.left), getShuffleCnt(smj.right))
736+
}.head
737+
assert(shuffleCnt._1 === 1)
738+
// Right child of join has 2 shuffle nodes because of Hybrid Scan for appended files.
739+
assert(shuffleCnt._2 === 2)
740+
741+
// Verify indexes are used, and all index files are picked.
742+
verifyIndexUsage(
743+
query,
744+
getIndexFilesPath(indexConfig.indexName, Seq(0)) ++ appendedFiles ++
745+
getIndexFilesPath(indexConfig2.indexName, Seq(0)) ++ appendedFiles)
746+
}
747+
withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED -> "true") {
748+
val execPlan = query().queryExecution.executedPlan
749+
val shuffleCnt = execPlan.collect {
750+
case smj: SortMergeJoinExec =>
751+
(getShuffleCnt(smj.left), getShuffleCnt(smj.right))
752+
}.head
753+
assert(shuffleCnt._1 === 1)
754+
// One shuffle node of right child is removed with shuffle count check.
755+
assert(shuffleCnt._2 === 1)
756+
757+
// For right child, indexRight can be still applied by FilterIndexRule.
758+
verifyIndexUsage(
759+
query,
760+
getIndexFilesPath(indexConfig.indexName, Seq(0)) ++ appendedFiles ++
761+
getIndexFilesPath(indexConfig2.indexName, Seq(0)) ++ appendedFiles)
762+
}
680763
}
681764
}
682765
}

0 commit comments

Comments
 (0)