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

Commit e082581

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

File tree

4 files changed

+166
-24
lines changed

4 files changed

+166
-24
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

+49-17
Original file line numberDiff line numberDiff line change
@@ -25,12 +25,15 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, Attribu
2525
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project}
2626
import org.apache.spark.sql.catalyst.rules.Rule
2727
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
28+
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
29+
import org.apache.spark.sql.execution.joins.SortMergeJoinExec
2830

2931
import com.microsoft.hyperspace.{ActiveSparkSession, Hyperspace}
3032
import com.microsoft.hyperspace.actions.Constants
3133
import com.microsoft.hyperspace.index._
3234
import com.microsoft.hyperspace.index.rankers.JoinIndexRanker
3335
import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEventLogging, HyperspaceIndexUsageEvent}
36+
import com.microsoft.hyperspace.util.HyperspaceConf
3437
import com.microsoft.hyperspace.util.ResolverUtils._
3538

3639
/**
@@ -68,18 +71,51 @@ object JoinIndexRule
6871
right =
6972
RuleUtils.transformPlanToUseIndex(spark, rIndex, r, useBucketSpec = true))
7073

71-
logEvent(
72-
HyperspaceIndexUsageEvent(
73-
AppInfo(
74-
sparkContext.sparkUser,
75-
sparkContext.applicationId,
76-
sparkContext.appName),
77-
Seq(lIndex, rIndex),
78-
join.toString,
79-
updatedPlan.toString,
80-
"Join index rule applied."))
81-
82-
updatedPlan
74+
val resultPlan =
75+
if (HyperspaceConf.hybridScanEnabled(spark) && HyperspaceConf
76+
.hybridScanShuffleCheckEnabled(spark)) {
77+
val execPlan = spark.sessionState.executePlan(updatedPlan).executedPlan
78+
val shuffleCnt = execPlan.collect {
79+
case smj: SortMergeJoinExec =>
80+
val leftShuffleCnt = smj.left.collect {
81+
case _: ShuffleExchangeExec => true
82+
}.length
83+
val rightShuffleCnt = smj.right.collect {
84+
case _: ShuffleExchangeExec => true
85+
}.length
86+
(leftShuffleCnt, rightShuffleCnt)
87+
}.head
88+
89+
// If the number of shuffle is 2, the candidate index pair cannot remove
90+
// the shuffles in both left and right for join and also Hybrid Scan causes
91+
// an additional shuffle for merging appended files.
92+
// We don't apply the index for the child with 2 shuffle nodes using Join Rule.
93+
// However, the child node is still applicable for Filter Rule.
94+
if (shuffleCnt._1 == 2) {
95+
updatedPlan.copy(left = l)
96+
} else if (shuffleCnt._2 == 2) {
97+
updatedPlan.copy(right = r)
98+
} else {
99+
updatedPlan
100+
}
101+
} else {
102+
updatedPlan
103+
}
104+
105+
if (!resultPlan.equals(join)) {
106+
logEvent(
107+
HyperspaceIndexUsageEvent(
108+
AppInfo(
109+
sparkContext.sparkUser,
110+
sparkContext.applicationId,
111+
sparkContext.appName),
112+
Seq(lIndex, rIndex),
113+
join.toString,
114+
updatedPlan.toString,
115+
"Join index rule applied."))
116+
}
117+
118+
resultPlan
83119
}
84120
.getOrElse(join)
85121
} catch {
@@ -325,11 +361,7 @@ object JoinIndexRule
325361
compatibleIndexPairs.map(
326362
indexPairs =>
327363
JoinIndexRanker
328-
.rank(
329-
spark,
330-
leftRel,
331-
rightRel,
332-
indexPairs)
364+
.rank(spark, leftRel, rightRel, indexPairs)
333365
.head)
334366
}
335367

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

+103-7
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
2323
import org.apache.spark.sql.execution.SortExec
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,108 @@ 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 with Shuffle checker 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+
val inputFiles = df.inputFiles
698+
699+
withSQLConf(IndexConstants.INDEX_NUM_BUCKETS -> "11") {
700+
// Create index.
701+
hyperspace.createIndex(df, indexConfig)
702+
}
703+
withSQLConf(IndexConstants.INDEX_NUM_BUCKETS -> "12") {
704+
// Create index.
705+
hyperspace.createIndex(df, indexConfig2)
706+
}
707+
708+
// Append to original data.
709+
SampleData.testData
710+
.toDF("c1", "c2", "c3", "c4", "c5")
711+
.limit(3)
712+
.write
713+
.mode("append")
714+
.parquet(testPath)
715+
716+
{
717+
// Create a join query.
718+
val leftDf = spark.read.parquet(testPath)
719+
val rightDf = spark.read.parquet(testPath)
720+
721+
def query(): DataFrame = {
722+
val filter1 = leftDf.select("c2", "c3")
723+
val filter2 = rightDf.select("c2", "c4")
724+
filter1
725+
.join(filter2, "c2")
726+
}
727+
728+
val appendedFiles = leftDf.inputFiles.diff(inputFiles).map(new Path(_))
729+
730+
spark.enableHyperspace()
731+
withSQLConf(TestConfig.HybridScanEnabled: _*) {
732+
withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED -> "false") {
733+
val execPlan = query().queryExecution.executedPlan
734+
val shuffleCnt = execPlan.collect {
735+
case smj: SortMergeJoinExec =>
736+
val leftShuffleCnt = smj.left.collect {
737+
case _: ShuffleExchangeExec => true
738+
}.length
739+
val rightShuffleCnt = smj.right.collect {
740+
case _: ShuffleExchangeExec => true
741+
}.length
742+
(leftShuffleCnt, rightShuffleCnt)
743+
}.head
744+
assert(shuffleCnt._1 === 1)
745+
// Right child of join has 2 shuffle node because of hybrid scan.
746+
assert(shuffleCnt._2 === 2)
747+
748+
// Verify indexes are used, and all index files are picked.
749+
verifyIndexUsage(
750+
query,
751+
getIndexFilesPath(indexConfig.indexName, Seq(0)) ++ appendedFiles ++
752+
getIndexFilesPath(indexConfig2.indexName, Seq(0)) ++ appendedFiles) // for Right
753+
}
754+
withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_SHUFFLE_CHECK_ENABLED -> "true") {
755+
val execPlan = query().queryExecution.executedPlan
756+
val shuffleCnt = execPlan.collect {
757+
case smj: SortMergeJoinExec =>
758+
val leftShuffleCnt = smj.left.collect {
759+
case _: ShuffleExchangeExec => true
760+
}.length
761+
val rightShuffleCnt = smj.right.collect {
762+
case _: ShuffleExchangeExec => true
763+
}.length
764+
(leftShuffleCnt, rightShuffleCnt)
765+
}.head
766+
assert(shuffleCnt._1 === 1)
767+
// One shuffle node of right child is removed by shuffle checker.
768+
assert(shuffleCnt._2 === 1)
769+
770+
// For right child, indexRight can be still applied by filter rule.
771+
verifyIndexUsage(
772+
query,
773+
getIndexFilesPath(indexConfig.indexName, Seq(0)) ++ appendedFiles ++
774+
getIndexFilesPath(indexConfig2.indexName, Seq(0)) ++ appendedFiles) // for Right
775+
}
680776
}
681777
}
682778
}

0 commit comments

Comments
 (0)