diff options
author | Cheng Lian <lian.cs.zju@gmail.com> | 2014-08-13 17:37:55 -0700 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2014-08-13 17:37:55 -0700 |
commit | 376a82e196e102ef49b9722e8be0b01ac5890a8b (patch) | |
tree | eb77a56236b577fd5129b7eff83b36a4d9b5e145 /sql | |
parent | 9256d4a9c8c9ddb9ae6bbe3c3b99b03fb66b946b (diff) | |
download | spark-376a82e196e102ef49b9722e8be0b01ac5890a8b.tar.gz spark-376a82e196e102ef49b9722e8be0b01ac5890a8b.tar.bz2 spark-376a82e196e102ef49b9722e8be0b01ac5890a8b.zip |
[SPARK-2650][SQL] More precise initial buffer size estimation for in-memory column buffer
This is a follow up of #1880.
Since the row number within a single batch is known, we can estimate a much more precise initial buffer size when building an in-memory column buffer.
Author: Cheng Lian <lian.cs.zju@gmail.com>
Closes #1901 from liancheng/precise-init-buffer-size and squashes the following commits:
d5501fa [Cheng Lian] More precise initial buffer size estimation for in-memory column buffer
Diffstat (limited to 'sql')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala | 11 |
1 files changed, 6 insertions, 5 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 3364d0e18b..e63b490304 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 @@ -20,12 +20,11 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{SparkPlan, LeafNode} -import org.apache.spark.sql.Row -import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.{LeafNode, SparkPlan} object InMemoryRelation { def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = @@ -48,7 +47,9 @@ private[sql] case class InMemoryRelation( new Iterator[Array[ByteBuffer]] { def next() = { val columnBuilders = output.map { attribute => - ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) + val columnType = ColumnType(attribute.dataType) + val initialBufferSize = columnType.defaultSize * batchSize + ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) }.toArray var row: Row = null |