File tree Expand file tree Collapse file tree 4 files changed +13
-14
lines changed
catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical
core/src/main/scala/org/apache/spark/sql
hive/src/main/scala/org/apache/spark/sql/hive Expand file tree Collapse file tree 4 files changed +13
-14
lines changed Original file line number Diff line number Diff line change @@ -26,18 +26,17 @@ import org.apache.spark.sql.catalyst.trees
26
26
abstract class LogicalPlan extends QueryPlan [LogicalPlan ] {
27
27
self : Product =>
28
28
29
- // TODO: make a case class?
30
29
/**
31
30
* Estimates of various statistics. The default estimation logic simply sums up the corresponding
32
31
* statistic produced by the children. To override this behavior, override `statistics` and
33
32
* assign it a overriden version of `Statistics`.
34
33
*/
35
- protected class Statistics {
36
- lazy val childrenStats = children.map(_.statistics)
37
- lazy val numTuples : Long = childrenStats.map(_.numTuples).sum
38
- lazy val sizeInBytes : Long = childrenStats.map(_.sizeInBytes).sum
39
- }
34
+ case class Statistics (
35
+ numTuples : Long = childrenStats.map(_.numTuples).sum,
36
+ sizeInBytes : Long = childrenStats.map(_.sizeInBytes).sum
37
+ )
40
38
lazy val statistics : Statistics = new Statistics
39
+ lazy val childrenStats = children.map(_.statistics)
41
40
42
41
/**
43
42
* Returns the set of attributes that are referenced by this node
Original file line number Diff line number Diff line change @@ -81,19 +81,19 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
81
81
}).asInstanceOf [this .type ]
82
82
}
83
83
84
- override lazy val statistics = new Statistics {
84
+ @ transient override lazy val statistics = Statistics (
85
85
// If this is wrapping around ExistingRdd and no reasonable estimation logic is implemented,
86
86
// return a default value.
87
- override lazy val sizeInBytes : Long = {
87
+ sizeInBytes = {
88
88
val defaultSum = childrenStats.map(_.sizeInBytes).sum
89
89
alreadyPlanned match {
90
90
// TODO: Instead of returning a default value here, find a way to return a meaningful
91
- // estimate for RDDs. See PR 1238 for more discussions.
91
+ // size estimate for RDDs. See PR 1238 for more discussions.
92
92
case e : ExistingRdd if defaultSum == 0 => statsDefaultSizeInBytes
93
93
case _ => defaultSum
94
94
}
95
95
}
96
- }
96
+ )
97
97
98
98
}
99
99
Original file line number Diff line number Diff line change @@ -53,14 +53,14 @@ private[sql] case class ParquetRelation(
53
53
54
54
self : Product =>
55
55
56
- @ transient override lazy val statistics = new Statistics {
56
+ @ transient override lazy val statistics = Statistics (
57
57
// TODO: investigate getting encoded column statistics in the parquet file?
58
- override lazy val sizeInBytes : Long = {
58
+ sizeInBytes = {
59
59
val hdfsPath = new Path (path)
60
60
val fs = hdfsPath.getFileSystem(conf.getOrElse(ContextUtil .getConfiguration(new Job ())))
61
61
fs.getContentSummary(hdfsPath).getLength // TODO: in bytes or system-dependent?
62
62
}
63
- }
63
+ )
64
64
65
65
/** Schema derived from ParquetFile */
66
66
def parquetSchema : MessageType =
Original file line number Diff line number Diff line change @@ -273,7 +273,7 @@ private[hive] case class MetastoreRelation
273
273
@ transient override lazy val statistics = new Statistics {
274
274
// TODO: check if this estimate is valid for tables after partition pruning.
275
275
// Size getters adapted from SizeBasedBigTableSelectorForAutoSMJ.java in Hive (version 0.13).
276
- override lazy val sizeInBytes : Long =
276
+ override val sizeInBytes : Long =
277
277
maybeGetSize(hiveConf, hiveQlTable.getProperty(" totalSize" ), path)
278
278
279
279
private [this ] def maybeGetSize (conf : HiveConf , size : String , path : Path ): Long = {
You can’t perform that action at this time.
0 commit comments