Skip to content

Commit 7a60ab7

Browse files
s/Estimates/Statistics, s/cardinality/numTuples.
1 parent de3ae13 commit 7a60ab7

File tree

5 files changed

+15
-15
lines changed

5 files changed

+15
-15
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -26,16 +26,16 @@ import org.apache.spark.sql.catalyst.trees
2626
abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
2727
self: Product =>
2828

29-
protected class Estimates {
30-
lazy val childrenEstimations = children.map(_.estimates)
31-
lazy val cardinality: Long = childrenEstimations.map(_.cardinality).sum
32-
lazy val sizeInBytes: Long = childrenEstimations.map(_.sizeInBytes).sum
29+
protected class Statistics {
30+
lazy val childrenStats = children.map(_.statistics)
31+
lazy val numTuples: Long = childrenStats.map(_.numTuples).sum
32+
lazy val sizeInBytes: Long = childrenStats.map(_.sizeInBytes).sum
3333
}
3434

3535
/**
3636
* Estimates of various statistics.
3737
*/
38-
lazy val estimates: Estimates = new Estimates
38+
lazy val statistics: Statistics = new Statistics
3939

4040
/**
4141
* Returns the set of attributes that are referenced by this node

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
6969
condition,
7070
left,
7171
right)
72-
if right.estimates.sizeInBytes <= sqlContext.autoConvertJoinSize =>
72+
if right.statistics.sizeInBytes <= sqlContext.autoConvertJoinSize =>
7373
broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight)
7474

7575
case ExtractEquiJoinKeys(
@@ -79,7 +79,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
7979
condition,
8080
left,
8181
right)
82-
if left.estimates.sizeInBytes <= sqlContext.autoConvertJoinSize =>
82+
if left.statistics.sizeInBytes <= sqlContext.autoConvertJoinSize =>
8383
broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft)
8484

8585
case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) =>
@@ -271,8 +271,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
271271
execution.Limit(limit, planLater(child))(sqlContext) :: Nil
272272
case Unions(unionChildren) =>
273273
execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil
274-
case logical.Except(left,right) =>
275-
execution.Except(planLater(left),planLater(right)) :: Nil
274+
case logical.Except(left,right) =>
275+
execution.Except(planLater(left),planLater(right)) :: Nil
276276
case logical.Intersect(left, right) =>
277277
execution.Intersect(planLater(left), planLater(right)) :: Nil
278278
case logical.Generate(generator, join, outer, _, child) =>

sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ private[sql] case class ParquetRelation(
5353

5454
self: Product =>
5555

56-
@transient override lazy val estimates = new Estimates {
56+
@transient override lazy val statistics = new Statistics {
5757
// TODO: investigate getting encoded column statistics in the parquet file?
5858
override lazy val sizeInBytes: Long = {
5959
val hdfsPath = new Path(path)

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -270,7 +270,7 @@ private[hive] case class MetastoreRelation
270270
}
271271

272272
// TODO: are there any stats in hiveQlTable.getSkewedInfo that we can use?
273-
@transient override lazy val estimates = new Estimates {
273+
@transient override lazy val statistics = new Statistics {
274274
// TODO: check if this estimate is valid for tables after partition pruning.
275275
// Size getters adapted from SizeBasedBigTableSelectorForAutoSMJ.java in Hive (version 0.13).
276276
override lazy val sizeInBytes: Long =

sql/hive/src/test/scala/org/apache/spark/sql/hive/EstimatesSuite.scala renamed to sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -25,14 +25,14 @@ import org.apache.spark.sql.hive.test.TestHive._
2525
import org.apache.spark.sql.parquet.{ParquetRelation, ParquetTestData}
2626
import org.apache.spark.util.Utils
2727

28-
class EstimatesSuite extends QueryTest {
28+
class StatisticsSuite extends QueryTest {
2929

3030
test("estimates the size of a test ParquetRelation") {
3131
ParquetTestData.writeFile()
3232
val testRDD = parquetFile(ParquetTestData.testDir.toString)
3333

3434
val sizes = testRDD.logicalPlan.collect { case j: ParquetRelation =>
35-
(j.estimates.sizeInBytes, j.newInstance.estimates.sizeInBytes)
35+
(j.statistics.sizeInBytes, j.newInstance.statistics.sizeInBytes)
3636
}
3737
assert(sizes.size === 1)
3838
assert(sizes(0)._1 == sizes(0)._2, "after .newInstance, estimates are different from before")
@@ -44,7 +44,7 @@ class EstimatesSuite extends QueryTest {
4444
test("estimates the size of a test MetastoreRelation") {
4545
val rdd = hql("""SELECT * FROM src""")
4646
val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation =>
47-
mr.estimates.sizeInBytes
47+
mr.statistics.sizeInBytes
4848
}
4949
assert(sizes.size === 1 && sizes(0) > 0)
5050
}
@@ -63,7 +63,7 @@ class EstimatesSuite extends QueryTest {
6363
// Assert src has a size smaller than the threshold.
6464
val sizes = rdd.queryExecution.analyzed.collect {
6565
case r if ct.runtimeClass.isAssignableFrom(r.getClass) =>
66-
r.estimates.sizeInBytes
66+
r.statistics.sizeInBytes
6767
}
6868
assert(sizes.size === 2 && sizes(0) <= autoConvertJoinSize,
6969
s"query should contain two relations, each of which has size smaller than autoConvertSize")

0 commit comments

Comments
 (0)