Skip to content

Commit 5b3a65a

Browse files
author
Marcelo Vanzin
committed
[SPARK-17549][sql] Coalesce cached relation stats in driver.
Currently there's a scalability problem with cached relations, in that stats for every column, for each partition, are captured in the driver. For large tables that leads to lots and lots of memory usage. This change modifies the accumulator used to capture stats in the driver to summarize the data as it arrives, instead of collecting everything and then summarizing it. Previously, for each column, the driver needed: (64 + 2 * sizeof(type)) * number of partitions With the change, the driver requires a fixed 8 bytes per column. On top of that, the change fixes a second problem dealing with how statistics of cached relations that share stats with another one (e.g. a cache projection of a cached relation) are calculated; previously, the data would be wrong since the accumulator data would be summarized based on the child output (while the data reflected the parent's output). Now the calculation is done based on how the child's output maps to the parent's output, yielding the correct size.
1 parent 7e418e9 commit 5b3a65a

File tree

2 files changed

+105
-16
lines changed

2 files changed

+105
-16
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala

Lines changed: 80 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical
3131
import org.apache.spark.sql.catalyst.plans.logical.Statistics
3232
import org.apache.spark.sql.execution.SparkPlan
3333
import org.apache.spark.storage.StorageLevel
34-
import org.apache.spark.util.CollectionAccumulator
34+
import org.apache.spark.util.AccumulatorV2
3535

3636

3737
object InMemoryRelation {
@@ -44,6 +44,70 @@ object InMemoryRelation {
4444
new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)()
4545
}
4646

47+
/**
48+
* Accumulator for storing column stats. Summarizes the data in the driver to curb the amount of
49+
* memory being used. Only "sizeInBytes" for each column is kept.
50+
*/
51+
class ColStatsAccumulator(originalOutput: Seq[Attribute])
52+
extends AccumulatorV2[Seq[ColumnStats], Array[Long]] {
53+
54+
private var stats: Array[Long] = null
55+
56+
override def isZero: Boolean = stats == null
57+
58+
override def copy(): AccumulatorV2[Seq[ColumnStats], Array[Long]] = {
59+
val newAcc = new ColStatsAccumulator(originalOutput)
60+
newAcc.stats = stats
61+
newAcc
62+
}
63+
64+
override def reset(): Unit = {
65+
stats = null
66+
}
67+
68+
override def add(update: Seq[ColumnStats]): Unit = {
69+
if (update != null) {
70+
require(isZero || stats.length == update.size, "Input stats doesn't match expected size.")
71+
72+
val newStats = new Array[Long](update.size)
73+
74+
update.toIndexedSeq.zipWithIndex.foreach { case (colStats, idx) =>
75+
val current = if (!isZero) stats(idx) else 0L
76+
newStats(idx) = current + colStats.sizeInBytes
77+
}
78+
79+
stats = newStats
80+
}
81+
}
82+
83+
override def merge(other: AccumulatorV2[Seq[ColumnStats], Array[Long]]): Unit = {
84+
if (other.value != null) {
85+
require(isZero || stats.length == other.value.length,
86+
"Merging accumulators of different size.")
87+
88+
val newStats = new Array[Long](other.value.length)
89+
for (i <- 0 until other.value.size) {
90+
val current = if (!isZero) stats(i) else 0L
91+
newStats(i) = current + other.value(i)
92+
}
93+
stats = newStats
94+
}
95+
}
96+
97+
override def value: Array[Long] = stats
98+
99+
/**
100+
* Calculate the size of the relation for a given output. Adds up all the known column sizes
101+
* that match the desired output.
102+
*/
103+
def sizeForOutput(output: Seq[Attribute]): Long = {
104+
originalOutput.toIndexedSeq.zipWithIndex.map { case (a, idx) =>
105+
val count = output.count(a.semanticEquals)
106+
stats(idx) * count
107+
}.fold(0L)(_ + _)
108+
}
109+
110+
}
47111

48112
/**
49113
* CachedBatch is a cached batch of rows.
@@ -63,8 +127,7 @@ case class InMemoryRelation(
63127
@transient child: SparkPlan,
64128
tableName: Option[String])(
65129
@transient var _cachedColumnBuffers: RDD[CachedBatch] = null,
66-
val batchStats: CollectionAccumulator[InternalRow] =
67-
child.sqlContext.sparkContext.collectionAccumulator[InternalRow])
130+
_batchStats: ColStatsAccumulator = null)
68131
extends logical.LeafNode with MultiInstanceRelation {
69132

70133
override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child)
@@ -73,22 +136,23 @@ case class InMemoryRelation(
73136

74137
@transient val partitionStatistics = new PartitionStatistics(output)
75138

139+
val batchStats = if (_batchStats != null) {
140+
_batchStats
141+
} else {
142+
val _newStats = new ColStatsAccumulator(output)
143+
child.sqlContext.sparkContext.register(_newStats)
144+
_newStats
145+
}
146+
76147
override lazy val statistics: Statistics = {
77-
if (batchStats.value.isEmpty) {
148+
if (batchStats.isZero) {
78149
// Underlying columnar RDD hasn't been materialized, no useful statistics information
79150
// available, return the default statistics.
80151
Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes)
81152
} else {
82153
// Underlying columnar RDD has been materialized, required information has also been
83154
// collected via the `batchStats` accumulator.
84-
val sizeOfRow: Expression =
85-
BindReferences.bindReference(
86-
output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add),
87-
partitionStatistics.schema)
88-
89-
val sizeInBytes =
90-
batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum
91-
Statistics(sizeInBytes = sizeInBytes)
155+
Statistics(sizeInBytes = batchStats.sizeForOutput(output))
92156
}
93157
}
94158

@@ -139,13 +203,13 @@ case class InMemoryRelation(
139203
rowCount += 1
140204
}
141205

142-
val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics)
143-
.flatMap(_.values))
144-
206+
val stats = columnBuilders.map(_.columnStats)
145207
batchStats.add(stats)
208+
209+
val statsRow = InternalRow.fromSeq(stats.map(_.collectedStatistics).flatMap(_.values))
146210
CachedBatch(rowCount, columnBuilders.map { builder =>
147211
JavaUtils.bufferToArray(builder.build())
148-
}, stats)
212+
}, statsRow)
149213
}
150214

151215
def hasNext: Boolean = rowIterator.hasNext

sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -232,4 +232,29 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
232232
val columnTypes2 = List.fill(length2)(IntegerType)
233233
val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2)
234234
}
235+
236+
test("SPARK-17549: cached table size should be correctly calculated") {
237+
val data = spark.sparkContext.parallelize(1 to 10, 5).map { i => (i, i.toLong) }
238+
.toDF("col1", "col2")
239+
val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
240+
val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None)
241+
242+
// Materialize the data.
243+
val expectedAnswer = data.collect()
244+
checkAnswer(cached, expectedAnswer)
245+
246+
// Check that the right size was calculated.
247+
val expectedColSizes = expectedAnswer.size * (INT.defaultSize + LONG.defaultSize)
248+
assert(cached.statistics.sizeInBytes === expectedColSizes)
249+
250+
// Create a projection of the cached data and make sure the statistics are correct.
251+
val projected = cached.withOutput(Seq(plan.output.last))
252+
assert(projected.statistics.sizeInBytes === expectedAnswer.size * LONG.defaultSize)
253+
254+
// Create a silly projection that repeats columns of the first cached relation, and
255+
// check that the size is calculated correctly.
256+
val projected2 = cached.withOutput(Seq(plan.output.last, plan.output.last))
257+
assert(projected2.statistics.sizeInBytes === 2 * expectedAnswer.size * LONG.defaultSize)
258+
}
259+
235260
}

0 commit comments

Comments
 (0)