@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow
2525import org .apache .spark .sql .catalyst .analysis .MultiInstanceRelation
2626import org .apache .spark .sql .catalyst .expressions ._
2727import org .apache .spark .sql .catalyst .plans .logical
28- import org .apache .spark .sql .catalyst .plans .logical .{HintInfo , Statistics }
28+ import org .apache .spark .sql .catalyst .plans .logical .{HintInfo , LogicalPlan , Statistics }
2929import org .apache .spark .sql .execution .SparkPlan
3030import org .apache .spark .storage .StorageLevel
3131import org .apache .spark .util .LongAccumulator
@@ -38,9 +38,9 @@ object InMemoryRelation {
3838 storageLevel : StorageLevel ,
3939 child : SparkPlan ,
4040 tableName : Option [String ],
41- statsOfPlanToCache : Statistics ): InMemoryRelation =
41+ logicalPlan : LogicalPlan ): InMemoryRelation =
4242 new InMemoryRelation (child.output, useCompression, batchSize, storageLevel, child, tableName)(
43- statsOfPlanToCache = statsOfPlanToCache )
43+ statsOfPlanToCache = logicalPlan.stats, outputOrdering = logicalPlan.outputOrdering )
4444}
4545
4646
@@ -63,7 +63,8 @@ case class InMemoryRelation(
6363 tableName : Option [String ])(
6464 @ transient var _cachedColumnBuffers : RDD [CachedBatch ] = null ,
6565 val sizeInBytesStats : LongAccumulator = child.sqlContext.sparkContext.longAccumulator,
66- statsOfPlanToCache : Statistics )
66+ statsOfPlanToCache : Statistics ,
67+ override val outputOrdering : Seq [SortOrder ])
6768 extends logical.LeafNode with MultiInstanceRelation {
6869
6970 override protected def innerChildren : Seq [SparkPlan ] = Seq (child)
@@ -149,7 +150,7 @@ case class InMemoryRelation(
149150 def withOutput (newOutput : Seq [Attribute ]): InMemoryRelation = {
150151 InMemoryRelation (
151152 newOutput, useCompression, batchSize, storageLevel, child, tableName)(
152- _cachedColumnBuffers, sizeInBytesStats, statsOfPlanToCache)
153+ _cachedColumnBuffers, sizeInBytesStats, statsOfPlanToCache, outputOrdering )
153154 }
154155
155156 override def newInstance (): this .type = {
@@ -162,13 +163,12 @@ case class InMemoryRelation(
162163 tableName)(
163164 _cachedColumnBuffers,
164165 sizeInBytesStats,
165- statsOfPlanToCache).asInstanceOf [this .type ]
166+ statsOfPlanToCache,
167+ outputOrdering).asInstanceOf [this .type ]
166168 }
167169
168170 def cachedColumnBuffers : RDD [CachedBatch ] = _cachedColumnBuffers
169171
170172 override protected def otherCopyArgs : Seq [AnyRef ] =
171173 Seq (_cachedColumnBuffers, sizeInBytesStats, statsOfPlanToCache)
172-
173- override def outputOrdering : Seq [SortOrder ] = child.outputOrdering
174174}
0 commit comments