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 d553bb6169ecc7612e9b8fc3062150aeff6263ca..45f15fd04d4e2e5341330d2352541ecb48964f87 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