@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
26
26
import org .apache .spark .sql .catalyst .expressions ._
27
27
import org .apache .spark .sql .catalyst .plans .QueryPlan
28
28
import org .apache .spark .sql .catalyst .plans .logical
29
- import org .apache .spark .sql .catalyst .plans .logical .{LogicalPlan , Statistics }
29
+ import org .apache .spark .sql .catalyst .plans .logical .{ColumnStat , LogicalPlan , Statistics }
30
30
import org .apache .spark .sql .catalyst .util .truncatedString
31
31
import org .apache .spark .sql .execution .SparkPlan
32
32
import org .apache .spark .storage .StorageLevel
@@ -145,37 +145,60 @@ object InMemoryRelation {
145
145
tableName : Option [String ],
146
146
logicalPlan : LogicalPlan ): InMemoryRelation = {
147
147
val cacheBuilder = CachedRDDBuilder (useCompression, batchSize, storageLevel, child, tableName)
148
- new InMemoryRelation (child.output, cacheBuilder, logicalPlan.outputOrdering)(
149
- statsOfPlanToCache = logicalPlan.stats)
148
+ val relation = new InMemoryRelation (child.output, cacheBuilder, logicalPlan.outputOrdering)
149
+ relation.statsOfPlanToCache = logicalPlan.stats
150
+ relation
150
151
}
151
152
152
153
def apply (cacheBuilder : CachedRDDBuilder , logicalPlan : LogicalPlan ): InMemoryRelation = {
153
- new InMemoryRelation (cacheBuilder.cachedPlan.output, cacheBuilder, logicalPlan.outputOrdering)(
154
- statsOfPlanToCache = logicalPlan.stats)
154
+ val relation = new InMemoryRelation (
155
+ cacheBuilder.cachedPlan.output, cacheBuilder, logicalPlan.outputOrdering)
156
+ relation.statsOfPlanToCache = logicalPlan.stats
157
+ relation
158
+ }
159
+
160
+ def apply (
161
+ output : Seq [Attribute ],
162
+ cacheBuilder : CachedRDDBuilder ,
163
+ outputOrdering : Seq [SortOrder ],
164
+ statsOfPlanToCache : Statistics ): InMemoryRelation = {
165
+ val relation = InMemoryRelation (output, cacheBuilder, outputOrdering)
166
+ relation.statsOfPlanToCache = statsOfPlanToCache
167
+ relation
155
168
}
156
169
}
157
170
158
171
case class InMemoryRelation (
159
172
output : Seq [Attribute ],
160
173
@ transient cacheBuilder : CachedRDDBuilder ,
161
- override val outputOrdering : Seq [SortOrder ])(
162
- @ volatile var statsOfPlanToCache : Statistics )
174
+ override val outputOrdering : Seq [SortOrder ])
163
175
extends logical.LeafNode with MultiInstanceRelation {
164
176
177
+ @ volatile var statsOfPlanToCache : Statistics = null
178
+
165
179
override protected def innerChildren : Seq [SparkPlan ] = Seq (cachedPlan)
166
180
167
181
override def doCanonicalize (): logical.LogicalPlan =
168
182
copy(output = output.map(QueryPlan .normalizeExprId(_, cachedPlan.output)),
169
183
cacheBuilder,
170
- outputOrdering)(
171
- statsOfPlanToCache)
184
+ outputOrdering)
172
185
173
186
override def producedAttributes : AttributeSet = outputSet
174
187
175
188
@ transient val partitionStatistics = new PartitionStatistics (output)
176
189
177
190
def cachedPlan : SparkPlan = cacheBuilder.cachedPlan
178
191
192
+ private [sql] def updateStats (
193
+ rowCount : Long ,
194
+ newColStats : Map [Attribute , ColumnStat ]): Unit = this .synchronized {
195
+ val newStats = statsOfPlanToCache.copy(
196
+ rowCount = Some (rowCount),
197
+ attributeStats = AttributeMap ((statsOfPlanToCache.attributeStats ++ newColStats).toSeq)
198
+ )
199
+ statsOfPlanToCache = newStats
200
+ }
201
+
179
202
override def computeStats (): Statistics = {
180
203
if (cacheBuilder.sizeInBytesStats.value == 0L ) {
181
204
// Underlying columnar RDD hasn't been materialized, use the stats from the plan to cache.
@@ -185,20 +208,17 @@ case class InMemoryRelation(
185
208
}
186
209
}
187
210
188
- def withOutput (newOutput : Seq [Attribute ]): InMemoryRelation = {
189
- InMemoryRelation (newOutput, cacheBuilder, outputOrdering)(statsOfPlanToCache)
190
- }
211
+ def withOutput (newOutput : Seq [Attribute ]): InMemoryRelation =
212
+ InMemoryRelation (newOutput, cacheBuilder, outputOrdering, statsOfPlanToCache)
191
213
192
214
override def newInstance (): this .type = {
193
- new InMemoryRelation (
215
+ InMemoryRelation (
194
216
output.map(_.newInstance()),
195
217
cacheBuilder,
196
- outputOrdering)(
197
- statsOfPlanToCache).asInstanceOf [this .type ]
218
+ outputOrdering,
219
+ statsOfPlanToCache).asInstanceOf [this .type ]
198
220
}
199
221
200
- override protected def otherCopyArgs : Seq [AnyRef ] = Seq (statsOfPlanToCache)
201
-
202
222
override def simpleString (maxFields : Int ): String =
203
223
s " InMemoryRelation [ ${truncatedString(output, " , " , maxFields)}], ${cacheBuilder.storageLevel}"
204
224
}
0 commit comments