@@ -32,19 +32,6 @@ import org.apache.spark.storage.StorageLevel
3232import  org .apache .spark .util .LongAccumulator 
3333
3434
35- object  InMemoryRelation  {
36-   def  apply (
37-       useCompression : Boolean ,
38-       batchSize : Int ,
39-       storageLevel : StorageLevel ,
40-       child : SparkPlan ,
41-       tableName : Option [String ],
42-       logicalPlan : LogicalPlan ):  InMemoryRelation  = 
43-     new  InMemoryRelation (child.output, useCompression, batchSize, storageLevel, child, tableName)(
44-       statsOfPlanToCache =  logicalPlan.stats, outputOrdering =  logicalPlan.outputOrdering)
45- }
46- 
47- 
4835/** 
4936 * CachedBatch is a cached batch of rows. 
5037 * 
@@ -55,58 +42,41 @@ object InMemoryRelation {
5542private [columnar]
5643case  class  CachedBatch (numRows : Int , buffers : Array [Array [Byte ]], stats : InternalRow )
5744
58- case  class  InMemoryRelation (
59-     output : Seq [Attribute ],
45+ case  class  CachedRDDBuilder (
6046    useCompression : Boolean ,
6147    batchSize : Int ,
6248    storageLevel : StorageLevel ,
63-     @ transient child : SparkPlan ,
49+     @ transient cachedPlan : SparkPlan ,
6450    tableName : Option [String ])(
65-     @ transient var  _cachedColumnBuffers :  RDD [CachedBatch ] =  null ,
66-     val  sizeInBytesStats :  LongAccumulator  =  child.sqlContext.sparkContext.longAccumulator,
67-     statsOfPlanToCache : Statistics ,
68-     override  val  outputOrdering :  Seq [SortOrder ])
69-   extends  logical.LeafNode  with  MultiInstanceRelation  {
70- 
71-   override  protected  def  innerChildren :  Seq [SparkPlan ] =  Seq (child)
72- 
73-   override  def  doCanonicalize ():  logical.LogicalPlan  = 
74-     copy(output =  output.map(QueryPlan .normalizeExprId(_, child.output)),
75-       storageLevel =  StorageLevel .NONE ,
76-       child =  child.canonicalized,
77-       tableName =  None )(
78-       _cachedColumnBuffers,
79-       sizeInBytesStats,
80-       statsOfPlanToCache,
81-       outputOrdering)
51+     @ transient private  var  _cachedColumnBuffers :  RDD [CachedBatch ] =  null ) {
8252
83-   override  def  producedAttributes :  AttributeSet  =  outputSet
84- 
85-   @ transient val  partitionStatistics  =  new  PartitionStatistics (output)
53+   val  sizeInBytesStats :  LongAccumulator  =  cachedPlan.sqlContext.sparkContext.longAccumulator
8654
87-   override  def  computeStats ():  Statistics  =  {
88-     if  (sizeInBytesStats.value ==  0L ) {
89-       //  Underlying columnar RDD hasn't been materialized, use the stats from the plan to cache.
90-       //  Note that we should drop the hint info here. We may cache a plan whose root node is a hint
91-       //  node. When we lookup the cache with a semantically same plan without hint info, the plan
92-       //  returned by cache lookup should not have hint info. If we lookup the cache with a
93-       //  semantically same plan with a different hint info, `CacheManager.useCachedData` will take
94-       //  care of it and retain the hint info in the lookup input plan.
95-       statsOfPlanToCache.copy(hints =  HintInfo ())
96-     } else  {
97-       Statistics (sizeInBytes =  sizeInBytesStats.value.longValue)
55+   def  cachedColumnBuffers :  RDD [CachedBatch ] =  {
56+     if  (_cachedColumnBuffers ==  null ) {
57+       synchronized  {
58+         if  (_cachedColumnBuffers ==  null ) {
59+           _cachedColumnBuffers =  buildBuffers()
60+         }
61+       }
9862    }
63+     _cachedColumnBuffers
9964  }
10065
101-   //  If the cached column buffers were not passed in, we calculate them in the constructor.
102-   //  As in Spark, the actual work of caching is lazy.
103-   if  (_cachedColumnBuffers ==  null ) {
104-     buildBuffers()
66+   def  clearCache (blocking : Boolean  =  true ):  Unit  =  {
67+     if  (_cachedColumnBuffers !=  null ) {
68+       synchronized  {
69+         if  (_cachedColumnBuffers !=  null ) {
70+           _cachedColumnBuffers.unpersist(blocking)
71+           _cachedColumnBuffers =  null 
72+         }
73+       }
74+     }
10575  }
10676
107-   private  def  buildBuffers ():  Unit  =  {
108-     val  output  =  child .output
109-     val  cached  =  child .execute().mapPartitionsInternal { rowIterator => 
77+   private  def  buildBuffers ():  RDD [ CachedBatch ]  =  {
78+     val  output  =  cachedPlan .output
79+     val  cached  =  cachedPlan .execute().mapPartitionsInternal { rowIterator => 
11080      new  Iterator [CachedBatch ] {
11181        def  next ():  CachedBatch  =  {
11282          val  columnBuilders  =  output.map { attribute => 
@@ -154,32 +124,77 @@ case class InMemoryRelation(
154124
155125    cached.setName(
156126      tableName.map(n =>  s " In-memory table  $n" )
157-         .getOrElse(StringUtils .abbreviate(child.toString, 1024 )))
158-     _cachedColumnBuffers =  cached
127+         .getOrElse(StringUtils .abbreviate(cachedPlan.toString, 1024 )))
128+     cached
129+   }
130+ }
131+ 
132+ object  InMemoryRelation  {
133+ 
134+   def  apply (
135+       useCompression : Boolean ,
136+       batchSize : Int ,
137+       storageLevel : StorageLevel ,
138+       child : SparkPlan ,
139+       tableName : Option [String ],
140+       logicalPlan : LogicalPlan ):  InMemoryRelation  =  {
141+     val  cacheBuilder  =  CachedRDDBuilder (useCompression, batchSize, storageLevel, child, tableName)()
142+     new  InMemoryRelation (child.output, cacheBuilder)(
143+       statsOfPlanToCache =  logicalPlan.stats, outputOrdering =  logicalPlan.outputOrdering)
144+   }
145+ 
146+   def  apply (cacheBuilder : CachedRDDBuilder , logicalPlan : LogicalPlan ):  InMemoryRelation  =  {
147+     new  InMemoryRelation (cacheBuilder.cachedPlan.output, cacheBuilder)(
148+       statsOfPlanToCache =  logicalPlan.stats, outputOrdering =  logicalPlan.outputOrdering)
149+   }
150+ }
151+ 
152+ case  class  InMemoryRelation (
153+     output : Seq [Attribute ],
154+     @ transient cacheBuilder : CachedRDDBuilder )(
155+     statsOfPlanToCache : Statistics ,
156+     override  val  outputOrdering :  Seq [SortOrder ])
157+   extends  logical.LeafNode  with  MultiInstanceRelation  {
158+ 
159+   override  protected  def  innerChildren :  Seq [SparkPlan ] =  Seq (cachedPlan)
160+ 
161+   override  def  doCanonicalize ():  logical.LogicalPlan  = 
162+     copy(output =  output.map(QueryPlan .normalizeExprId(_, cachedPlan.output)),
163+       cacheBuilder)(
164+       statsOfPlanToCache,
165+       outputOrdering)
166+ 
167+   override  def  producedAttributes :  AttributeSet  =  outputSet
168+ 
169+   @ transient val  partitionStatistics  =  new  PartitionStatistics (output)
170+ 
171+   def  cachedPlan :  SparkPlan  =  cacheBuilder.cachedPlan
172+ 
173+   override  def  computeStats ():  Statistics  =  {
174+     if  (cacheBuilder.sizeInBytesStats.value ==  0L ) {
175+       //  Underlying columnar RDD hasn't been materialized, use the stats from the plan to cache.
176+       //  Note that we should drop the hint info here. We may cache a plan whose root node is a hint
177+       //  node. When we lookup the cache with a semantically same plan without hint info, the plan
178+       //  returned by cache lookup should not have hint info. If we lookup the cache with a
179+       //  semantically same plan with a different hint info, `CacheManager.useCachedData` will take
180+       //  care of it and retain the hint info in the lookup input plan.
181+       statsOfPlanToCache.copy(hints =  HintInfo ())
182+     } else  {
183+       Statistics (sizeInBytes =  cacheBuilder.sizeInBytesStats.value.longValue)
184+     }
159185  }
160186
161187  def  withOutput (newOutput : Seq [Attribute ]):  InMemoryRelation  =  {
162-     InMemoryRelation (
163-       newOutput, useCompression, batchSize, storageLevel, child, tableName)(
164-         _cachedColumnBuffers, sizeInBytesStats, statsOfPlanToCache, outputOrdering)
188+     InMemoryRelation (newOutput, cacheBuilder)(statsOfPlanToCache, outputOrdering)
165189  }
166190
167191  override  def  newInstance ():  this .type  =  {
168192    new  InMemoryRelation (
169193      output.map(_.newInstance()),
170-       useCompression,
171-       batchSize,
172-       storageLevel,
173-       child,
174-       tableName)(
175-         _cachedColumnBuffers,
176-         sizeInBytesStats,
194+       cacheBuilder)(
177195        statsOfPlanToCache,
178196        outputOrdering).asInstanceOf [this .type ]
179197  }
180198
181-   def  cachedColumnBuffers :  RDD [CachedBatch ] =  _cachedColumnBuffers
182- 
183-   override  protected  def  otherCopyArgs :  Seq [AnyRef ] = 
184-     Seq (_cachedColumnBuffers, sizeInBytesStats, statsOfPlanToCache)
199+   override  protected  def  otherCopyArgs :  Seq [AnyRef ] =  Seq (statsOfPlanToCache)
185200}
0 commit comments