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 @@ -41,9 +41,14 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
case class Statistics(
sizeInBytes: BigInt
)
lazy val statistics: Statistics = Statistics(
sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product
)
lazy val statistics: Statistics = {
if (children.size == 0) {
throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.")
}

Statistics(
sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product)
}

/**
* Returns the set of attributes that this node takes as
Expand Down Expand Up @@ -117,9 +122,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
*/
abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] {
self: Product =>

override lazy val statistics: Statistics =
throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.")
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,8 +89,10 @@ class SQLContext(@transient val sparkContext: SparkContext)
*
* @group userf
*/
implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) =
implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = {
SparkPlan.currentContext.set(self)
new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))(self))
}

/**
* :: DeveloperApi ::
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,9 @@ private[sql] case class InMemoryRelation(
(private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null)
extends LogicalPlan with MultiInstanceRelation {

override lazy val statistics =
Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes)

// 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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
* populated by the query planning infrastructure.
*/
@transient
protected val sqlContext = SparkPlan.currentContext.get()
protected[spark] val sqlContext = SparkPlan.currentContext.get()

protected def sparkContext = sqlContext.sparkContext

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,14 @@ class InMemoryColumnarQuerySuite extends QueryTest {
checkAnswer(scan, testData.collect().toSeq)
}

test("default size avoids broadcast") {
// TODO: Improve this test when we have better statistics
sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).registerTempTable("sizeTst")
cacheTable("sizeTst")
assert(
table("sizeTst").queryExecution.logical.statistics.sizeInBytes > autoBroadcastJoinThreshold)
}

test("projection") {
val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan
val scan = InMemoryRelation(useCompression = true, 5, plan)
Expand Down