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

Commit 63dfc53

Browse files
committed
Filter index file paths if bucket pruning is applicable
1 parent 9ddf44b commit 63dfc53

File tree

3 files changed

+236
-16
lines changed

3 files changed

+236
-16
lines changed

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

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import com.microsoft.hyperspace.Hyperspace
3232
import com.microsoft.hyperspace.index._
3333
import com.microsoft.hyperspace.index.IndexLogEntryTags.{HYBRIDSCAN_RELATED_CONFIGS, IS_HYBRIDSCAN_CANDIDATE}
3434
import com.microsoft.hyperspace.index.plans.logical.{BucketUnion, IndexHadoopFsRelation}
35-
import com.microsoft.hyperspace.util.HyperspaceConf
35+
import com.microsoft.hyperspace.util.{HyperspaceConf, LogicalPlanUtils}
3636

3737
object RuleUtils {
3838

@@ -273,8 +273,9 @@ object RuleUtils {
273273
// Project(A,B) -> Filter(C = 10) -> Index Scan (A,B,C)
274274
plan transformDown {
275275
case baseRelation @ LogicalRelation(_: HadoopFsRelation, baseOutput, _, _) =>
276+
val indexFiles = LogicalPlanUtils.indexFilesSelectedBuecktdOnly(plan, index)
276277
val location = index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY) {
277-
new InMemoryFileIndex(spark, index.content.files, Map(), None)
278+
new InMemoryFileIndex(spark, indexFiles, Map(), None)
278279
}
279280

280281
val relation = new IndexHadoopFsRelation(
@@ -353,6 +354,8 @@ object RuleUtils {
353354
}
354355
}
355356

357+
val indexFiles = LogicalPlanUtils.indexFilesSelectedBuecktdOnly(plan, index)
358+
356359
val filesToRead = {
357360
if (useBucketSpec || !index.hasParquetAsSourceFormat || filesDeleted.nonEmpty ||
358361
location.partitionSchema.nonEmpty) {
@@ -367,12 +370,12 @@ object RuleUtils {
367370
// If the source relation is partitioned, we cannot read the appended files with the
368371
// index data as the schema of partitioned files are not equivalent to the index data.
369372
unhandledAppendedFiles = filesAppended
370-
index.content.files
373+
indexFiles
371374
} else {
372375
// If BucketSpec of index data isn't used (e.g., in the case of FilterIndex currently)
373376
// and the source format is parquet, we could read the appended files along
374377
// with the index data.
375-
index.content.files ++ filesAppended
378+
indexFiles ++ filesAppended
376379
}
377380
}
378381

@@ -386,7 +389,7 @@ object RuleUtils {
386389

387390
def fileIndex: InMemoryFileIndex =
388391
new InMemoryFileIndex(spark, filesToRead, Map(), None)
389-
val newLocation = if (filesToRead.length == index.content.files.size) {
392+
val newLocation = if (filesToRead.length == indexFiles.size) {
390393
index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY)(fileIndex)
391394
} else {
392395
index.withCachedTag(plan, IndexLogEntryTags.INMEMORYFILEINDEX_HYBRID_SCAN)(fileIndex)

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

Lines changed: 138 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,17 @@
1616

1717
package com.microsoft.hyperspace.util
1818

19+
import org.apache.hadoop.fs.Path
20+
import org.apache.spark.sql.catalyst.catalog.BucketSpec
21+
import org.apache.spark.sql.catalyst.expressions
22+
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, EmptyRow, Expression, Literal, SubqueryExpression}
23+
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
1924
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
20-
import org.apache.spark.sql.execution.datasources.LogicalRelation
25+
import org.apache.spark.sql.execution.datasources.{BucketingUtils, HadoopFsRelation, LogicalRelation}
26+
import org.apache.spark.sql.types.{DoubleType, FloatType}
27+
import org.apache.spark.util.collection.BitSet
28+
29+
import com.microsoft.hyperspace.index.IndexLogEntry
2130

2231
/**
2332
* Utility functions for logical plan.
@@ -35,4 +44,132 @@ object LogicalPlanUtils {
3544
case _ => false
3645
}
3746
}
47+
48+
/**
49+
* BucketSelector returns the selected buckets if bucket pruning is applicable for the given
50+
* query plan. The logic is extracted from [[FileSourceScanStrategy]] in Spark.
51+
*/
52+
object BucketSelector {
53+
// should prune buckets iff num buckets is greater than 1 and there is only one bucket column
54+
private def shouldPruneBuckets(spec: BucketSpec): Boolean = {
55+
spec.bucketColumnNames.length == 1 && spec.numBuckets > 1
56+
}
57+
58+
private def getExpressionBuckets(
59+
expr: Expression,
60+
bucketColumnName: String,
61+
numBuckets: Int): BitSet = {
62+
63+
def getBucketNumber(attr: Attribute, v: Any): Int = {
64+
BucketingUtils.getBucketIdFromValue(attr, numBuckets, v)
65+
}
66+
67+
def getBucketSetFromIterable(attr: Attribute, iter: Iterable[Any]): BitSet = {
68+
val matchedBuckets = new BitSet(numBuckets)
69+
iter
70+
.map(v => getBucketNumber(attr, v))
71+
.foreach(bucketNum => matchedBuckets.set(bucketNum))
72+
matchedBuckets
73+
}
74+
75+
def getBucketSetFromValue(attr: Attribute, v: Any): BitSet = {
76+
val matchedBuckets = new BitSet(numBuckets)
77+
matchedBuckets.set(getBucketNumber(attr, v))
78+
matchedBuckets
79+
}
80+
81+
expr match {
82+
case expressions.Equality(a: Attribute, Literal(v, _)) if a.name == bucketColumnName =>
83+
getBucketSetFromValue(a, v)
84+
case expressions.In(a: Attribute, list)
85+
if list.forall(_.isInstanceOf[Literal]) && a.name == bucketColumnName =>
86+
getBucketSetFromIterable(a, list.map(e => e.eval(EmptyRow)))
87+
case expressions.InSet(a: Attribute, hset) if a.name == bucketColumnName =>
88+
getBucketSetFromIterable(a, hset)
89+
case expressions.IsNull(a: Attribute) if a.name == bucketColumnName =>
90+
getBucketSetFromValue(a, null)
91+
case expressions.IsNaN(a: Attribute)
92+
if a.name == bucketColumnName && a.dataType == FloatType =>
93+
getBucketSetFromValue(a, Float.NaN)
94+
case expressions.IsNaN(a: Attribute)
95+
if a.name == bucketColumnName && a.dataType == DoubleType =>
96+
getBucketSetFromValue(a, Double.NaN)
97+
case expressions.And(left, right) =>
98+
getExpressionBuckets(left, bucketColumnName, numBuckets) &
99+
getExpressionBuckets(right, bucketColumnName, numBuckets)
100+
case expressions.Or(left, right) =>
101+
getExpressionBuckets(left, bucketColumnName, numBuckets) |
102+
getExpressionBuckets(right, bucketColumnName, numBuckets)
103+
case _ =>
104+
val matchedBuckets = new BitSet(numBuckets)
105+
matchedBuckets.setUntil(numBuckets)
106+
matchedBuckets
107+
}
108+
}
109+
110+
private def genBucketSet(
111+
normalizedFilters: Seq[Expression],
112+
bucketSpec: BucketSpec): Option[BitSet] = {
113+
if (normalizedFilters.isEmpty) {
114+
return None
115+
}
116+
117+
val bucketColumnName = bucketSpec.bucketColumnNames.head
118+
val numBuckets = bucketSpec.numBuckets
119+
120+
val normalizedFiltersAndExpr = normalizedFilters
121+
.reduce(expressions.And)
122+
val matchedBuckets =
123+
getExpressionBuckets(normalizedFiltersAndExpr, bucketColumnName, numBuckets)
124+
125+
val numBucketsSelected = matchedBuckets.cardinality()
126+
127+
// None means all the buckets need to be scanned
128+
if (numBucketsSelected == numBuckets) {
129+
None
130+
} else {
131+
Some(matchedBuckets)
132+
}
133+
}
134+
135+
def apply(plan: LogicalPlan, bucketSpec: BucketSpec): Option[BitSet] = plan match {
136+
case PhysicalOperation(
137+
projects,
138+
filters,
139+
l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _)) =>
140+
// The attribute name of predicate could be different than the one in schema in case of
141+
// case insensitive, we should change them to match the one in schema, so we do not need to
142+
// worry about case sensitivity anymore.
143+
val normalizedFilters = filters.map { e =>
144+
e transform {
145+
case a: AttributeReference =>
146+
a.withName(l.output.find(_.semanticEquals(a)).get.name)
147+
}
148+
}
149+
// subquery expressions are filtered out because they can't be used to prune buckets or
150+
// pushed down as data filters, yet they would be executed
151+
val normalizedFiltersWithoutSubqueries =
152+
normalizedFilters.filterNot(SubqueryExpression.hasSubquery)
153+
154+
val bucketSet = if (shouldPruneBuckets(bucketSpec)) {
155+
genBucketSet(normalizedFiltersWithoutSubqueries, bucketSpec)
156+
} else {
157+
None
158+
}
159+
bucketSet
160+
case _ => None
161+
}
162+
}
163+
164+
private[hyperspace] def indexFilesSelectedBuecktdOnly(
165+
plan: LogicalPlan,
166+
index: IndexLogEntry): Seq[Path] = {
167+
BucketSelector(plan, index.bucketSpec) match {
168+
case Some(selectedBuckets) =>
169+
index.content.files.filter(f =>
170+
selectedBuckets.get(BucketingUtils.getBucketId(f.getName).get))
171+
case None =>
172+
index.content.files
173+
}
174+
}
38175
}

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

Lines changed: 90 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ 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}
2323
import org.apache.spark.sql.execution.SortExec
24-
import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, InMemoryFileIndex, LogicalRelation}
24+
import org.apache.spark.sql.execution.datasources.{BucketingUtils, FileIndex, HadoopFsRelation, InMemoryFileIndex, LogicalRelation}
2525
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
2626

2727
import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData, TestConfig, TestUtils}
@@ -30,6 +30,7 @@ import com.microsoft.hyperspace.index.IndexLogEntryTags._
3030
import com.microsoft.hyperspace.index.execution.BucketUnionStrategy
3131
import com.microsoft.hyperspace.index.plans.logical.IndexHadoopFsRelation
3232
import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule}
33+
import com.microsoft.hyperspace.util.LogicalPlanUtils.BucketSelector
3334
import com.microsoft.hyperspace.util.PathUtils
3435

3536
class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
@@ -112,7 +113,11 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
112113

113114
def query(): DataFrame = df.filter("c3 == 'facebook'").select("c3", "c1")
114115

115-
verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName))
116+
verifyIndexUsage(
117+
query,
118+
getIndexFilesPathWithBucketSelector(
119+
query().queryExecution.optimizedPlan,
120+
indexConfig.indexName))
116121
}
117122
}
118123
}
@@ -128,7 +133,11 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
128133
def query(): DataFrame = df.filter("C3 == 'facebook'").select("C3", "c1")
129134

130135
// Verify if case-insensitive index works with case-insensitive query.
131-
verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName))
136+
verifyIndexUsage(
137+
query,
138+
getIndexFilesPathWithBucketSelector(
139+
query().queryExecution.optimizedPlan,
140+
indexConfig.indexName))
132141
}
133142

134143
test("E2E test for case sensitive filter query where changing conf changes behavior.") {
@@ -145,7 +154,11 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
145154
}
146155

147156
withSQLConf("spark.sql.caseSensitive" -> "false") {
148-
verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName))
157+
verifyIndexUsage(
158+
query,
159+
getIndexFilesPathWithBucketSelector(
160+
query().queryExecution.optimizedPlan,
161+
indexConfig.indexName))
149162
}
150163
}
151164

@@ -165,9 +178,12 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
165178
def query(): DataFrame = spark.sql("SELECT * from t where c4 = 1")
166179

167180
// Verify no Project node is present in the query plan, as a result of using SELECT *
168-
assert(query().queryExecution.optimizedPlan.collect { case p: Project => p }.isEmpty)
181+
val queryPlan = query().queryExecution.optimizedPlan
182+
assert(queryPlan.collect { case p: Project => p }.isEmpty)
169183

170-
verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName))
184+
verifyIndexUsage(
185+
query,
186+
getIndexFilesPathWithBucketSelector(queryPlan, indexConfig.indexName))
171187
}
172188
}
173189
}
@@ -388,10 +404,11 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
388404

389405
spark.enableHyperspace()
390406
val dfWithHyperspaceEnabled = query(df)
407+
val planWithHyperspaceEnabled = dfWithHyperspaceEnabled.queryExecution.optimizedPlan
391408

392409
verifyQueryPlanHasExpectedRootPaths(
393-
dfWithHyperspaceEnabled.queryExecution.optimizedPlan,
394-
getIndexFilesPath(indexConfig.indexName))
410+
planWithHyperspaceEnabled,
411+
getIndexFilesPathWithBucketSelector(planWithHyperspaceEnabled, indexConfig.indexName))
395412

396413
assert(schemaWithHyperspaceDisabled.equals(dfWithHyperspaceEnabled.schema))
397414
assert(sortedRowsWithHyperspaceDisabled.sameElements(getSortedRows(dfWithHyperspaceEnabled)))
@@ -503,7 +520,11 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
503520
def query(): DataFrame =
504521
spark.read.parquet(testPath).filter("c3 == 'facebook'").select("c3", "c1")
505522

506-
verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName))
523+
verifyIndexUsage(
524+
query,
525+
getIndexFilesPathWithBucketSelector(
526+
query().queryExecution.optimizedPlan,
527+
indexConfig.indexName))
507528

508529
// Delete some source data file.
509530
TestUtils.deleteFiles(testPath, "*parquet", 1)
@@ -518,7 +539,12 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
518539
hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)
519540

520541
// Verify index usage on latest version of index (v=1) after refresh.
521-
verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName, Seq(1)))
542+
verifyIndexUsage(
543+
query,
544+
getIndexFilesPathWithBucketSelector(
545+
query().queryExecution.optimizedPlan,
546+
indexConfig.indexName,
547+
Seq(1)))
522548
}
523549
}
524550
}
@@ -951,6 +977,47 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
951977
}
952978
}
953979

980+
test("Verify excluding index data file path using bucket pruning.") {
981+
withTempPathAsString { testPath =>
982+
// Setup. Create data.
983+
val indexConfig = IndexConfig("index", Seq("c3"), Seq("c4"))
984+
import spark.implicits._
985+
SampleData.testData
986+
.toDF("c1", "c2", "c3", "c4", "c5")
987+
.limit(10)
988+
.write
989+
.json(testPath)
990+
val df = spark.read.json(testPath)
991+
992+
// Create index.
993+
hyperspace.createIndex(df, indexConfig)
994+
spark.enableHyperspace()
995+
996+
def query(): DataFrame =
997+
df.filter(df("c3") isin (Seq("facebook", "donde", "miperro"): _*)).select("c4", "c3")
998+
999+
withIndex("index") {
1000+
val index = TestUtils.latestIndexLogEntry(systemPath, indexConfig.indexName)
1001+
val plan = query().queryExecution.optimizedPlan
1002+
val buckets = BucketSelector(plan, index.bucketSpec)
1003+
assert(buckets.isDefined)
1004+
1005+
val locs = getFsLocation(plan)
1006+
assert(locs.size == 1)
1007+
assert(buckets.get.cardinality() == 3)
1008+
1009+
val indexFiles = locs.head.inputFiles
1010+
assert(indexFiles.length == buckets.get.cardinality())
1011+
assert(indexFiles.length < index.content.files.length)
1012+
1013+
val indexFilesBitIdSet = indexFiles.map(BucketingUtils.getBucketId(_).get).toSet
1014+
indexFilesBitIdSet.forall(buckets.get.get(_))
1015+
(1 to index.bucketSpec.numBuckets).forall(n =>
1016+
!(buckets.get.get(n) ^ indexFilesBitIdSet.contains(n)))
1017+
}
1018+
}
1019+
}
1020+
9541021
/**
9551022
* Verify that the query plan has the expected rootPaths.
9561023
*
@@ -980,6 +1047,19 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite {
9801047
}.flatten
9811048
}
9821049

1050+
private def getIndexFilesPathWithBucketSelector(
1051+
plan: LogicalPlan,
1052+
indexName: String,
1053+
versions: Seq[Int] = Seq(0)): Seq[Path] = {
1054+
val paths = getIndexFilesPath(indexName, versions)
1055+
BucketSelector(plan, TestUtils.latestIndexLogEntry(systemPath, indexName).bucketSpec) match {
1056+
case Some(buckets) =>
1057+
paths.filter(f => buckets.get(BucketingUtils.getBucketId(f.getName).get))
1058+
case None =>
1059+
paths
1060+
}
1061+
}
1062+
9831063
private def getIndexFilesPath(indexName: String, versions: Seq[Int] = Seq(0)): Seq[Path] = {
9841064
versions.flatMap { v =>
9851065
Content

0 commit comments

Comments
 (0)