Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
* of itself with globally unique expression ids.
*/
trait MultiInstanceRelation {
def newInstance: this.type
def newInstance(): this.type
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@

package org.apache.spark.sql.columnar

import java.nio.ByteBuffer

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
Expand All @@ -26,22 +29,19 @@ import org.apache.spark.SparkConf

object InMemoryRelation {
def apply(useCompression: Boolean, child: SparkPlan): InMemoryRelation =
new InMemoryRelation(child.output, useCompression, child)
new InMemoryRelation(child.output, useCompression, child)()
}

private[sql] case class InMemoryRelation(
output: Seq[Attribute],
useCompression: Boolean,
child: SparkPlan)
(private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null)
extends LogicalPlan with MultiInstanceRelation {

override def children = Seq.empty
override def references = Set.empty

override def newInstance() =
new InMemoryRelation(output.map(_.newInstance), useCompression, child).asInstanceOf[this.type]

lazy val cachedColumnBuffers = {
// If the cached column buffers were not passed in, we calculate them in the constructor.
// As in Spark, the actual work of caching is lazy.
if (_cachedColumnBuffers == null) {
val output = child.output
val cached = child.execute().mapPartitions { iterator =>
val columnBuilders = output.map { attribute =>
Expand All @@ -62,10 +62,23 @@ private[sql] case class InMemoryRelation(
}.cache()

cached.setName(child.toString)
// Force the materialization of the cached RDD.
cached.count()
cached
_cachedColumnBuffers = cached
}


override def children = Seq.empty

override def references = Set.empty

override def newInstance() = {
new InMemoryRelation(
output.map(_.newInstance),
useCompression,
child)(
_cachedColumnBuffers).asInstanceOf[this.type]
}

def cachedColumnBuffers = _cachedColumnBuffers
}

private[sql] case class InMemoryColumnarTableScan(
Expand Down