Github user cloud-fan commented on a diff in the pull request:
https://github.com/apache/spark/pull/10942#discussion_r51392868
--- Diff:
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +61,159 @@ class BucketedReadSuite extends QueryTest with
SQLTestUtils with TestHiveSinglet
}
}
+ // To verify if the bucket pruning works, this function checks two
conditions:
+ // 1) Check if the pruned buckets (before filtering) are empty.
+ // 2) Verify the final result is the same as the expected one
+ private def checkPrunedAnswers(
+ bucketSpec: BucketSpec,
+ bucketValues: Seq[Integer],
+ bucketedDataFrame: DataFrame,
+ expectedAnswer: DataFrame): Unit = {
+
+ val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec
+ // Limit: bucket pruning only works when the bucket column has one and
only one column
+ assert(bucketColumnNames.length == 1)
+ val bucketColumnIndex =
bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head)
+ val bucketColumn = bucketedDataFrame.schema.toAttributes.head
+ val matchedBuckets = new BitSet(numBuckets)
+ bucketValues.foreach { value =>
+ matchedBuckets.set(DataSourceStrategy.getBucketId(bucketColumn,
numBuckets, value))
+ }
+
+ // Filter could hide the bug in bucket pruning. Thus, skipping all the
filters
+ val rdd =
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+ assert(rdd.isDefined)
+
+ val checkBucketId = rdd.get.execute().map(_.copy()).mapPartitions(iter
=> {
+ iter.map(row =>
--- End diff --
I think we don't need to check every row here, but only partition id:
```
val checkedResult = rdd.get.execute().mapPartitionsWithIndex {
case (index, iter) => if (index == should be prunned bucket id)
Iterator(iter.isEmpty) else Iterator(true)
}
assert(checkedResult.collect().forall(_ == true))
```
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]