@@ -20,7 +20,7 @@ import org.apache.hadoop.conf.Configuration
2020import org .apache .hadoop .fs .Path
2121import org .apache .spark .sql .{AnalysisException , DataFrame , QueryTest , Row }
2222import org .apache .spark .sql .catalyst .plans .logical .{LogicalPlan , Project }
23- import org .apache .spark .sql .execution .SortExec
23+ import org .apache .spark .sql .execution .{ FileSourceScanExec , SortExec , UnionExec }
2424import org .apache .spark .sql .execution .datasources .{HadoopFsRelation , InMemoryFileIndex , LogicalRelation }
2525import org .apache .spark .sql .execution .exchange .ShuffleExchangeExec
2626
@@ -579,6 +579,112 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
579579 }
580580 }
581581
582+ test(" Verify adaptive bucket spec application for FilterIndexRule." ) {
583+ withTempPathAsString { testPath =>
584+ // Setup. Create data.
585+ val indexConfig = IndexConfig (" index" , Seq (" c3" ), Seq (" c4" ))
586+ import spark .implicits ._
587+ SampleData .testData
588+ .toDF(" c1" , " c2" , " c3" , " c4" , " c5" )
589+ .limit(10 )
590+ .write
591+ .json(testPath)
592+ val df = spark.read.json(testPath)
593+
594+ // Create index.
595+ hyperspace.createIndex(df, indexConfig)
596+ spark.enableHyperspace()
597+
598+ def query (): DataFrame =
599+ df.filter(df(" c3" ) isin (Seq (" facebook" , " donde" , " miperro" ): _* )).select(" c4" , " c3" )
600+
601+ withIndex(" index" ) {
602+ withSQLConf(" spark.sql.optimizer.inSetConversionThreshold" -> " 10" ) {
603+ // Check bucketSpec is applied.
604+ val execPlan = query.queryExecution.executedPlan
605+ val foundPrunedBuckets = execPlan.collect {
606+ case _ @ FileSourceScanExec (_, _, _, _, optionalBucketSet, _, _) =>
607+ optionalBucketSet.get.cardinality()
608+ }
609+ assert(foundPrunedBuckets.length == 1 )
610+ assert(foundPrunedBuckets.head == 3 )
611+
612+ verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName, Seq (0 )))
613+ }
614+
615+ // TODO: because of SPARK-33372, bucket pruning is not applied for InSet operator.
616+ // As indexes are bucketed, supporting bucket pruning can improve the performance of
617+ // queries with high selectivity. Will add a new FileSourceScanStrategy soon.
618+ withSQLConf(" spark.sql.optimizer.inSetConversionThreshold" -> " 2" ) {
619+ val execPlan = query().queryExecution.executedPlan
620+ // Check bucketSpec is not applied.
621+ val foundPrunedBuckets = execPlan.collect {
622+ case _ @ FileSourceScanExec (_, _, _, _, optionalBucketSet, _, _)
623+ if (optionalBucketSet.isDefined) =>
624+ optionalBucketSet.get.cardinality()
625+ }
626+ assert(foundPrunedBuckets.isEmpty)
627+ verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName, Seq (0 )))
628+ }
629+
630+ // Append to original data.
631+ SampleData .testData
632+ .toDF(" c1" , " c2" , " c3" , " c4" , " c5" )
633+ .limit(3 )
634+ .write
635+ .mode(" append" )
636+ .json(testPath)
637+
638+ val df2 = spark.read.json(testPath)
639+ val inputFiles = df.inputFiles
640+ val appendedFiles = df2.inputFiles.diff(inputFiles).map(new Path (_))
641+ def query2 (): DataFrame = {
642+ df2.filter(df2(" c3" ) isin (Seq (" facebook" , " donde" , " miperro" ): _* )).select(" c4" , " c3" )
643+ }
644+
645+ withSQLConf(TestConfig .HybridScanEnabled : _* ) {
646+ withSQLConf(" spark.sql.optimizer.inSetConversionThreshold" -> " 10" ) {
647+ // Check bucketSpec is applied.
648+ val execPlan = query2().queryExecution.executedPlan
649+ val foundPrunedBuckets = execPlan.collect {
650+ case _ @ UnionExec (children) =>
651+ val p = children.head.collect {
652+ case _ @ FileSourceScanExec (_, _, _, _, optionalBucketSet, _, _)
653+ if optionalBucketSet.isDefined =>
654+ optionalBucketSet.get.cardinality()
655+ }
656+ p.head
657+ }
658+ assert(foundPrunedBuckets.length == 1 )
659+ assert(foundPrunedBuckets.head == 3 )
660+ verifyIndexUsage(
661+ query2,
662+ getIndexFilesPath(indexConfig.indexName, Seq (0 )) ++ appendedFiles.toSeq)
663+ }
664+
665+ withSQLConf(" spark.sql.optimizer.inSetConversionThreshold" -> " 2" ) {
666+ val execPlan = query2().queryExecution.executedPlan
667+ // Check bucketSpec is not applied.
668+ val foundPrunedBuckets = execPlan.collect {
669+ case _ @ UnionExec (children) =>
670+ val b = children.head.collect {
671+ case _ @ FileSourceScanExec (_, _, _, _, optionalBucketSet, _, _)
672+ if optionalBucketSet.isDefined =>
673+ optionalBucketSet.get.cardinality()
674+ }
675+ assert(b.isEmpty)
676+ true
677+ }
678+ assert(foundPrunedBuckets.length == 1 )
679+ verifyIndexUsage(
680+ query2,
681+ getIndexFilesPath(indexConfig.indexName, Seq (0 )) ++ appendedFiles.toSeq)
682+ }
683+ }
684+ }
685+ }
686+ }
687+
582688 test(
583689 " Verify JoinIndexRule utilizes indexes correctly after quick refresh when some file " +
584690 " gets deleted and some appended to source data." ) {
@@ -670,13 +776,13 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
670776
671777 // Refreshed index as quick mode can be applied with Hybrid Scan config.
672778 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)
779+ spark.disableHyperspace()
780+ val dfWithHyperspaceDisabled = query()
781+ val basePlan = dfWithHyperspaceDisabled.queryExecution.optimizedPlan
782+ spark.enableHyperspace()
783+ val dfWithHyperspaceEnabled = query()
784+ assert(! basePlan.equals(dfWithHyperspaceEnabled.queryExecution.optimizedPlan))
785+ checkAnswer(dfWithHyperspaceDisabled, dfWithHyperspaceEnabled)
680786 }
681787 }
682788 }
0 commit comments