Skip to content

Commit

Permalink
[SPARK-3294][SQL] Eliminates boxing costs from in-memory columnar sto…
Browse files Browse the repository at this point in the history
…rage

This is a major refactoring of the in-memory columnar storage implementation, aims to eliminate boxing costs from critical paths (building/accessing column buffers) as much as possible. The basic idea is to refactor all major interfaces into a row-based form and use them together with `SpecificMutableRow`. The difficult part is how to adapt all compression schemes, esp. `RunLengthEncoding` and `DictionaryEncoding`, to this design. Since in-memory compression is disabled by default for now, and this PR should be strictly better than before no matter in-memory compression is enabled or not, maybe I'll finish that part in another PR.

**UPDATE** This PR also took the chance to optimize `HiveTableScan` by

1. leveraging `SpecificMutableRow` to avoid boxing cost, and
1. building specific `Writable` unwrapper functions a head of time to avoid per row pattern matching and branching costs.

TODO

- [x] Benchmark
- [ ] ~~Eliminate boxing costs in `RunLengthEncoding`~~ (left to future PRs)
- [ ] ~~Eliminate boxing costs in `DictionaryEncoding` (seems not easy to do without specializing `DictionaryEncoding` for every supported column type)~~  (left to future PRs)

## Micro benchmark

The benchmark uses a 10 million line CSV table consists of bytes, shorts, integers, longs, floats and doubles, measures the time to build the in-memory version of this table, and the time to scan the whole in-memory table.

Benchmark code can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-hivetablescanbenchmark-scala). Script used to generate the input table can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-tablegen-scala).

Speedup:

- Hive table scanning + column buffer building: **18.74%**

  The original benchmark uses 1K as in-memory batch size, when increased to 10K, it can be 28.32% faster.

- In-memory table scanning: **7.95%**

Before:

        | Building | Scanning
------- | -------- | --------
1       | 16472    | 525
2       | 16168    | 530
3       | 16386    | 529
4       | 16184    | 538
5       | 16209    | 521
Average | 16283.8  | 528.6

After:

        | Building | Scanning
------- | -------- | --------
1       | 13124    | 458
2       | 13260    | 529
3       | 12981    | 463
4       | 13214    | 483
5       | 13583    | 500
Average | 13232.4  | 486.6

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2327 from liancheng/prevent-boxing/unboxing and squashes the following commits:

4419fe4 [Cheng Lian] Addressing comments
e5d2cf2 [Cheng Lian] Bug fix: should call setNullAt when field value is null to avoid NPE
8b8552b [Cheng Lian] Only checks for partition batch pruning flag once
489f97b [Cheng Lian] Bug fix: TableReader.fillObject uses wrong ordinals
97bbc4e [Cheng Lian] Optimizes hive.TableReader by by providing specific Writable unwrappers a head of time
3dc1f94 [Cheng Lian] Minor changes to eliminate row object creation
5b39cb9 [Cheng Lian] Lowers log level of compression scheme details
f2a7890 [Cheng Lian] Use SpecificMutableRow in InMemoryColumnarTableScan to avoid boxing
9cf30b0 [Cheng Lian] Added row based ColumnType.append/extract
456c366 [Cheng Lian] Made compression decoder row based
edac3cd [Cheng Lian] Makes ColumnAccessor.extractSingle row based
8216936 [Cheng Lian] Removes boxing cost in IntDelta and LongDelta by providing specialized implementations
b70d519 [Cheng Lian] Made some in-memory columnar storage interfaces row-based
  • Loading branch information
liancheng authored and marmbrus committed Sep 13, 2014
1 parent 184cd51 commit 7404924
Show file tree
Hide file tree
Showing 24 changed files with 554 additions and 292 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ final class MutableByte extends MutableValue {
}

final class MutableAny extends MutableValue {
var value: Any = 0
var value: Any = _
def boxed = if (isNull) null else value
def update(v: Any) = value = {
isNull = false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,11 +50,13 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](

def hasNext = buffer.hasRemaining

def extractTo(row: MutableRow, ordinal: Int) {
columnType.setField(row, ordinal, extractSingle(buffer))
def extractTo(row: MutableRow, ordinal: Int): Unit = {
extractSingle(row, ordinal)
}

def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer)
def extractSingle(row: MutableRow, ordinal: Int): Unit = {
columnType.extract(buffer, row, ordinal)
}

protected def underlyingBuffer = buffer
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,10 +68,9 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType](
buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId)
}

override def appendFrom(row: Row, ordinal: Int) {
val field = columnType.getField(row, ordinal)
buffer = ensureFreeSpace(buffer, columnType.actualSize(field))
columnType.append(field, buffer)
override def appendFrom(row: Row, ordinal: Int): Unit = {
buffer = ensureFreeSpace(buffer, columnType.actualSize(row, ordinal))
columnType.append(row, ordinal, buffer)
}

override def build() = {
Expand Down Expand Up @@ -142,16 +141,16 @@ private[sql] object ColumnBuilder {
useCompression: Boolean = false): ColumnBuilder = {

val builder = (typeId match {
case INT.typeId => new IntColumnBuilder
case LONG.typeId => new LongColumnBuilder
case FLOAT.typeId => new FloatColumnBuilder
case DOUBLE.typeId => new DoubleColumnBuilder
case BOOLEAN.typeId => new BooleanColumnBuilder
case BYTE.typeId => new ByteColumnBuilder
case SHORT.typeId => new ShortColumnBuilder
case STRING.typeId => new StringColumnBuilder
case BINARY.typeId => new BinaryColumnBuilder
case GENERIC.typeId => new GenericColumnBuilder
case INT.typeId => new IntColumnBuilder
case LONG.typeId => new LongColumnBuilder
case FLOAT.typeId => new FloatColumnBuilder
case DOUBLE.typeId => new DoubleColumnBuilder
case BOOLEAN.typeId => new BooleanColumnBuilder
case BYTE.typeId => new ByteColumnBuilder
case SHORT.typeId => new ShortColumnBuilder
case STRING.typeId => new StringColumnBuilder
case BINARY.typeId => new BinaryColumnBuilder
case GENERIC.typeId => new GenericColumnBuilder
case TIMESTAMP.typeId => new TimestampColumnBuilder
}).asInstanceOf[ColumnBuilder]

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ private[sql] class ByteColumnStats extends ColumnStats {
var lower = Byte.MaxValue
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getByte(ordinal)
if (value > upper) upper = value
Expand All @@ -87,7 +87,7 @@ private[sql] class ShortColumnStats extends ColumnStats {
var lower = Short.MaxValue
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getShort(ordinal)
if (value > upper) upper = value
Expand All @@ -105,7 +105,7 @@ private[sql] class LongColumnStats extends ColumnStats {
var lower = Long.MaxValue
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getLong(ordinal)
if (value > upper) upper = value
Expand All @@ -123,7 +123,7 @@ private[sql] class DoubleColumnStats extends ColumnStats {
var lower = Double.MaxValue
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getDouble(ordinal)
if (value > upper) upper = value
Expand All @@ -141,7 +141,7 @@ private[sql] class FloatColumnStats extends ColumnStats {
var lower = Float.MaxValue
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getFloat(ordinal)
if (value > upper) upper = value
Expand All @@ -159,7 +159,7 @@ private[sql] class IntColumnStats extends ColumnStats {
var lower = Int.MaxValue
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getInt(ordinal)
if (value > upper) upper = value
Expand All @@ -177,7 +177,7 @@ private[sql] class StringColumnStats extends ColumnStats {
var lower: String = null
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row.getString(ordinal)
if (upper == null || value.compareTo(upper) > 0) upper = value
Expand All @@ -195,7 +195,7 @@ private[sql] class TimestampColumnStats extends ColumnStats {
var lower: Timestamp = null
var nullCount = 0

override def gatherStats(row: Row, ordinal: Int) {
override def gatherStats(row: Row, ordinal: Int): Unit = {
if (!row.isNullAt(ordinal)) {
val value = row(ordinal).asInstanceOf[Timestamp]
if (upper == null || value.compareTo(upper) > 0) upper = value
Expand Down
Loading

0 comments on commit 7404924

Please sign in to comment.