diff options
author | Kun Xu <viper_kun@163.com> | 2015-08-16 14:44:23 +0800 |
---|---|---|
committer | Cheng Lian <lian@databricks.com> | 2015-08-16 14:44:45 +0800 |
commit | 182f9b7a6d3a3ee7ec7de6abc24e296aa794e4e8 (patch) | |
tree | b74f4c822e198c33fedfe27fcb7ecfd23c181dd8 /sql/core | |
parent | 1db7179fae672fcec7b8de12c374dd384ce51c67 (diff) | |
download | spark-182f9b7a6d3a3ee7ec7de6abc24e296aa794e4e8.tar.gz spark-182f9b7a6d3a3ee7ec7de6abc24e296aa794e4e8.tar.bz2 spark-182f9b7a6d3a3ee7ec7de6abc24e296aa794e4e8.zip |
[SPARK-9973] [SQL] Correct in-memory columnar buffer size
The `initialSize` argument of `ColumnBuilder.initialize()` should be the
number of rows rather than bytes. However `InMemoryColumnarTableScan`
passes in a byte size, which makes Spark SQL allocate more memory than
necessary when building in-memory columnar buffers.
Author: Kun Xu <viper_kun@163.com>
Closes #8189 from viper-kun/errorSize.
Diffstat (limited to 'sql/core')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala | 3 |
1 files changed, 1 insertions, 2 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index d553bb6169..45f15fd04d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -121,8 +121,7 @@ private[sql] case class InMemoryRelation( def next(): CachedBatch = { val columnBuilders = output.map { attribute => val columnType = ColumnType(attribute.dataType) - val initialBufferSize = columnType.defaultSize * batchSize - ColumnBuilder(attribute.dataType, initialBufferSize, attribute.name, useCompression) + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray var rowCount = 0 |