@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
23
23
import org .apache .spark .sql .execution .SortExec
24
24
import org .apache .spark .sql .execution .datasources .{HadoopFsRelation , InMemoryFileIndex , LogicalRelation }
25
25
import org .apache .spark .sql .execution .exchange .ShuffleExchangeExec
26
+ import org .apache .spark .sql .execution .joins .SortMergeJoinExec
26
27
27
28
import com .microsoft .hyperspace .{Hyperspace , Implicits , SampleData , TestConfig , TestUtils }
28
29
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 {
670
671
671
672
// Refreshed index as quick mode can be applied with Hybrid Scan config.
672
673
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
+ }
680
776
}
681
777
}
682
778
}
0 commit comments