Skip to content
Snippets Groups Projects
Commit 1a7d7cc8 authored by Michael Armbrust's avatar Michael Armbrust Committed by Reynold Xin
Browse files

[SPARK-2405][SQL] Reusue same byte buffers when creating new instance of InMemoryRelation

Reuse byte buffers when creating unique attributes for multiple instances of an InMemoryRelation in a single query plan.

Author: Michael Armbrust <michael@databricks.com>

Closes #1332 from marmbrus/doubleCache and squashes the following commits:

4a19609 [Michael Armbrust] Clean up concurrency story by calculating buffersn the constructor.
b39c931 [Michael Armbrust] Allocations are kind of a side effect.
f67eff7 [Michael Armbrust] Reusue same byte buffers when creating new instance of InMemoryRelation
parent 7e26b576
No related branches found
No related tags found
No related merge requests found
...@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan ...@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
* of itself with globally unique expression ids. * of itself with globally unique expression ids.
*/ */
trait MultiInstanceRelation { trait MultiInstanceRelation {
def newInstance: this.type def newInstance(): this.type
} }
/** /**
......
...@@ -17,6 +17,9 @@ ...@@ -17,6 +17,9 @@
package org.apache.spark.sql.columnar 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.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
...@@ -26,22 +29,19 @@ import org.apache.spark.SparkConf ...@@ -26,22 +29,19 @@ import org.apache.spark.SparkConf
object InMemoryRelation { object InMemoryRelation {
def apply(useCompression: Boolean, child: SparkPlan): 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( private[sql] case class InMemoryRelation(
output: Seq[Attribute], output: Seq[Attribute],
useCompression: Boolean, useCompression: Boolean,
child: SparkPlan) child: SparkPlan)
(private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null)
extends LogicalPlan with MultiInstanceRelation { extends LogicalPlan with MultiInstanceRelation {
override def children = Seq.empty // If the cached column buffers were not passed in, we calculate them in the constructor.
override def references = Set.empty // As in Spark, the actual work of caching is lazy.
if (_cachedColumnBuffers == null) {
override def newInstance() =
new InMemoryRelation(output.map(_.newInstance), useCompression, child).asInstanceOf[this.type]
lazy val cachedColumnBuffers = {
val output = child.output val output = child.output
val cached = child.execute().mapPartitions { iterator => val cached = child.execute().mapPartitions { iterator =>
val columnBuilders = output.map { attribute => val columnBuilders = output.map { attribute =>
...@@ -62,10 +62,23 @@ private[sql] case class InMemoryRelation( ...@@ -62,10 +62,23 @@ private[sql] case class InMemoryRelation(
}.cache() }.cache()
cached.setName(child.toString) cached.setName(child.toString)
// Force the materialization of the cached RDD. _cachedColumnBuffers = cached
cached.count()
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( private[sql] case class InMemoryColumnarTableScan(
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment