Skip to content

Commit 1faa579

Browse files
lianchengpwendell
authored andcommitted
[SPARK-1371][WIP] Compression support for Spark SQL in-memory columnar storage
JIRA issue: [SPARK-1373](https://issues.apache.org/jira/browse/SPARK-1373) (Although tagged as WIP, this PR is structurally complete. The only things left unimplemented are 3 more compression algorithms: `BooleanBitSet`, `IntDelta` and `LongDelta`, which are trivial to add later in this or another separate PR.) This PR contains compression support for Spark SQL in-memory columnar storage. Main interfaces include: * `CompressionScheme` Each `CompressionScheme` represents a concrete compression algorithm, which basically consists of an `Encoder` for compression and a `Decoder` for decompression. Algorithms implemented include: * `RunLengthEncoding` * `DictionaryEncoding` Algorithms to be implemented include: * `BooleanBitSet` * `IntDelta` * `LongDelta` * `CompressibleColumnBuilder` A stackable `ColumnBuilder` trait used to build byte buffers for compressible columns. A best `CompressionScheme` that exhibits lowest compression ratio is chosen for each column according to statistical information gathered while elements are appended into the `ColumnBuilder`. However, if no `CompressionScheme` can achieve a compression ratio better than 80%, no compression will be done for this column to save CPU time. Memory layout of the final byte buffer is showed below: ``` .--------------------------- Column type ID (4 bytes) | .----------------------- Null count N (4 bytes) | | .------------------- Null positions (4 x N bytes, empty if null count is zero) | | | .------------- Compression scheme ID (4 bytes) | | | | .--------- Compressed non-null elements V V V V V +---+---+-----+---+---------+ | | | ... | | ... ... | +---+---+-----+---+---------+ \-----------/ \-----------/ header body ``` * `CompressibleColumnAccessor` A stackable `ColumnAccessor` trait used to iterate (possibly) compressed data column. * `ColumnStats` Used to collect statistical information while loading data into in-memory columnar table. Optimizations like partition pruning rely on this information. Strictly speaking, `ColumnStats` related code is not part of the compression support. It's contained in this PR to ensure and validate the row-based API design (which is used to avoid boxing/unboxing cost whenever possible). A major refactoring change since PR #205 is: * Refactored all getter/setter methods for primitive types in various places into `ColumnType` classes to remove duplicated code. Author: Cheng Lian <[email protected]> Closes #285 from liancheng/memColumnarCompression and squashes the following commits: ed71bbd [Cheng Lian] Addressed all PR comments by @marmbrus d3a4fa9 [Cheng Lian] Removed Ordering[T] in ColumnStats for better performance 5034453 [Cheng Lian] Bug fix, more tests, and more refactoring c298b76 [Cheng Lian] Test suites refactored 2780d6a [Cheng Lian] [WIP] in-memory columnar compression support 211331c [Cheng Lian] WIP: in-memory columnar compression support 85cc59b [Cheng Lian] Refactored ColumnAccessors & ColumnBuilders to remove duplicate code
1 parent 7823633 commit 1faa579

21 files changed

+1644
-408
lines changed

sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala

Lines changed: 24 additions & 79 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.nio.{ByteOrder, ByteBuffer}
2121

2222
import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType}
2323
import org.apache.spark.sql.catalyst.expressions.MutableRow
24-
import org.apache.spark.sql.execution.SparkSqlSerializer
24+
import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor
2525

2626
/**
2727
* An `Iterator` like trait used to extract values from columnar byte buffer. When a value is
@@ -41,121 +41,66 @@ private[sql] trait ColumnAccessor {
4141
protected def underlyingBuffer: ByteBuffer
4242
}
4343

44-
private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](buffer: ByteBuffer)
44+
private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](
45+
protected val buffer: ByteBuffer,
46+
protected val columnType: ColumnType[T, JvmType])
4547
extends ColumnAccessor {
4648

4749
protected def initialize() {}
4850

49-
def columnType: ColumnType[T, JvmType]
50-
5151
def hasNext = buffer.hasRemaining
5252

5353
def extractTo(row: MutableRow, ordinal: Int) {
54-
doExtractTo(row, ordinal)
54+
columnType.setField(row, ordinal, extractSingle(buffer))
5555
}
5656

57-
protected def doExtractTo(row: MutableRow, ordinal: Int)
57+
def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer)
5858

5959
protected def underlyingBuffer = buffer
6060
}
6161

6262
private[sql] abstract class NativeColumnAccessor[T <: NativeType](
63-
buffer: ByteBuffer,
64-
val columnType: NativeColumnType[T])
65-
extends BasicColumnAccessor[T, T#JvmType](buffer)
63+
override protected val buffer: ByteBuffer,
64+
override protected val columnType: NativeColumnType[T])
65+
extends BasicColumnAccessor(buffer, columnType)
6666
with NullableColumnAccessor
67+
with CompressibleColumnAccessor[T]
6768

6869
private[sql] class BooleanColumnAccessor(buffer: ByteBuffer)
69-
extends NativeColumnAccessor(buffer, BOOLEAN) {
70-
71-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
72-
row.setBoolean(ordinal, columnType.extract(buffer))
73-
}
74-
}
70+
extends NativeColumnAccessor(buffer, BOOLEAN)
7571

7672
private[sql] class IntColumnAccessor(buffer: ByteBuffer)
77-
extends NativeColumnAccessor(buffer, INT) {
78-
79-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
80-
row.setInt(ordinal, columnType.extract(buffer))
81-
}
82-
}
73+
extends NativeColumnAccessor(buffer, INT)
8374

8475
private[sql] class ShortColumnAccessor(buffer: ByteBuffer)
85-
extends NativeColumnAccessor(buffer, SHORT) {
86-
87-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
88-
row.setShort(ordinal, columnType.extract(buffer))
89-
}
90-
}
76+
extends NativeColumnAccessor(buffer, SHORT)
9177

9278
private[sql] class LongColumnAccessor(buffer: ByteBuffer)
93-
extends NativeColumnAccessor(buffer, LONG) {
94-
95-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
96-
row.setLong(ordinal, columnType.extract(buffer))
97-
}
98-
}
79+
extends NativeColumnAccessor(buffer, LONG)
9980

10081
private[sql] class ByteColumnAccessor(buffer: ByteBuffer)
101-
extends NativeColumnAccessor(buffer, BYTE) {
102-
103-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
104-
row.setByte(ordinal, columnType.extract(buffer))
105-
}
106-
}
82+
extends NativeColumnAccessor(buffer, BYTE)
10783

10884
private[sql] class DoubleColumnAccessor(buffer: ByteBuffer)
109-
extends NativeColumnAccessor(buffer, DOUBLE) {
110-
111-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
112-
row.setDouble(ordinal, columnType.extract(buffer))
113-
}
114-
}
85+
extends NativeColumnAccessor(buffer, DOUBLE)
11586

11687
private[sql] class FloatColumnAccessor(buffer: ByteBuffer)
117-
extends NativeColumnAccessor(buffer, FLOAT) {
118-
119-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
120-
row.setFloat(ordinal, columnType.extract(buffer))
121-
}
122-
}
88+
extends NativeColumnAccessor(buffer, FLOAT)
12389

12490
private[sql] class StringColumnAccessor(buffer: ByteBuffer)
125-
extends NativeColumnAccessor(buffer, STRING) {
126-
127-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
128-
row.setString(ordinal, columnType.extract(buffer))
129-
}
130-
}
91+
extends NativeColumnAccessor(buffer, STRING)
13192

13293
private[sql] class BinaryColumnAccessor(buffer: ByteBuffer)
133-
extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer)
134-
with NullableColumnAccessor {
135-
136-
def columnType = BINARY
137-
138-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
139-
row(ordinal) = columnType.extract(buffer)
140-
}
141-
}
94+
extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY)
95+
with NullableColumnAccessor
14296

14397
private[sql] class GenericColumnAccessor(buffer: ByteBuffer)
144-
extends BasicColumnAccessor[DataType, Array[Byte]](buffer)
145-
with NullableColumnAccessor {
146-
147-
def columnType = GENERIC
148-
149-
override protected def doExtractTo(row: MutableRow, ordinal: Int) {
150-
val serialized = columnType.extract(buffer)
151-
row(ordinal) = SparkSqlSerializer.deserialize[Any](serialized)
152-
}
153-
}
98+
extends BasicColumnAccessor[DataType, Array[Byte]](buffer, GENERIC)
99+
with NullableColumnAccessor
154100

155101
private[sql] object ColumnAccessor {
156-
def apply(b: ByteBuffer): ColumnAccessor = {
157-
// The first 4 bytes in the buffer indicates the column type.
158-
val buffer = b.duplicate().order(ByteOrder.nativeOrder())
102+
def apply(buffer: ByteBuffer): ColumnAccessor = {
103+
// The first 4 bytes in the buffer indicate the column type.
159104
val columnTypeId = buffer.getInt()
160105

161106
columnTypeId match {

sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala

Lines changed: 44 additions & 81 deletions
Original file line numberDiff line numberDiff line change
@@ -22,45 +22,52 @@ import java.nio.{ByteBuffer, ByteOrder}
2222
import org.apache.spark.sql.Row
2323
import org.apache.spark.sql.catalyst.types._
2424
import org.apache.spark.sql.columnar.ColumnBuilder._
25-
import org.apache.spark.sql.execution.SparkSqlSerializer
25+
import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder}
2626

2727
private[sql] trait ColumnBuilder {
2828
/**
2929
* Initializes with an approximate lower bound on the expected number of elements in this column.
3030
*/
3131
def initialize(initialSize: Int, columnName: String = "")
3232

33+
/**
34+
* Appends `row(ordinal)` to the column builder.
35+
*/
3336
def appendFrom(row: Row, ordinal: Int)
3437

38+
/**
39+
* Column statistics information
40+
*/
41+
def columnStats: ColumnStats[_, _]
42+
43+
/**
44+
* Returns the final columnar byte buffer.
45+
*/
3546
def build(): ByteBuffer
3647
}
3748

38-
private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder {
49+
private[sql] class BasicColumnBuilder[T <: DataType, JvmType](
50+
val columnStats: ColumnStats[T, JvmType],
51+
val columnType: ColumnType[T, JvmType])
52+
extends ColumnBuilder {
3953

40-
private var columnName: String = _
41-
protected var buffer: ByteBuffer = _
54+
protected var columnName: String = _
4255

43-
def columnType: ColumnType[T, JvmType]
56+
protected var buffer: ByteBuffer = _
4457

4558
override def initialize(initialSize: Int, columnName: String = "") = {
4659
val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize
4760
this.columnName = columnName
48-
buffer = ByteBuffer.allocate(4 + 4 + size * columnType.defaultSize)
61+
62+
// Reserves 4 bytes for column type ID
63+
buffer = ByteBuffer.allocate(4 + size * columnType.defaultSize)
4964
buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId)
5065
}
5166

52-
// Have to give a concrete implementation to make mixin possible
5367
override def appendFrom(row: Row, ordinal: Int) {
54-
doAppendFrom(row, ordinal)
55-
}
56-
57-
// Concrete `ColumnBuilder`s can override this method to append values
58-
protected def doAppendFrom(row: Row, ordinal: Int)
59-
60-
// Helper method to append primitive values (to avoid boxing cost)
61-
protected def appendValue(v: JvmType) {
62-
buffer = ensureFreeSpace(buffer, columnType.actualSize(v))
63-
columnType.append(v, buffer)
68+
val field = columnType.getField(row, ordinal)
69+
buffer = ensureFreeSpace(buffer, columnType.actualSize(field))
70+
columnType.append(field, buffer)
6471
}
6572

6673
override def build() = {
@@ -69,83 +76,39 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends C
6976
}
7077
}
7178

72-
private[sql] abstract class NativeColumnBuilder[T <: NativeType](
73-
val columnType: NativeColumnType[T])
74-
extends BasicColumnBuilder[T, T#JvmType]
79+
private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType](
80+
columnType: ColumnType[T, JvmType])
81+
extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType)
7582
with NullableColumnBuilder
7683

77-
private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(BOOLEAN) {
78-
override def doAppendFrom(row: Row, ordinal: Int) {
79-
appendValue(row.getBoolean(ordinal))
80-
}
81-
}
82-
83-
private[sql] class IntColumnBuilder extends NativeColumnBuilder(INT) {
84-
override def doAppendFrom(row: Row, ordinal: Int) {
85-
appendValue(row.getInt(ordinal))
86-
}
87-
}
84+
private[sql] abstract class NativeColumnBuilder[T <: NativeType](
85+
override val columnStats: NativeColumnStats[T],
86+
override val columnType: NativeColumnType[T])
87+
extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType)
88+
with NullableColumnBuilder
89+
with AllCompressionSchemes
90+
with CompressibleColumnBuilder[T]
8891

89-
private[sql] class ShortColumnBuilder extends NativeColumnBuilder(SHORT) {
90-
override def doAppendFrom(row: Row, ordinal: Int) {
91-
appendValue(row.getShort(ordinal))
92-
}
93-
}
92+
private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN)
9493

95-
private[sql] class LongColumnBuilder extends NativeColumnBuilder(LONG) {
96-
override def doAppendFrom(row: Row, ordinal: Int) {
97-
appendValue(row.getLong(ordinal))
98-
}
99-
}
94+
private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT)
10095

101-
private[sql] class ByteColumnBuilder extends NativeColumnBuilder(BYTE) {
102-
override def doAppendFrom(row: Row, ordinal: Int) {
103-
appendValue(row.getByte(ordinal))
104-
}
105-
}
96+
private[sql] class ShortColumnBuilder extends NativeColumnBuilder(new ShortColumnStats, SHORT)
10697

107-
private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(DOUBLE) {
108-
override def doAppendFrom(row: Row, ordinal: Int) {
109-
appendValue(row.getDouble(ordinal))
110-
}
111-
}
98+
private[sql] class LongColumnBuilder extends NativeColumnBuilder(new LongColumnStats, LONG)
11299

113-
private[sql] class FloatColumnBuilder extends NativeColumnBuilder(FLOAT) {
114-
override def doAppendFrom(row: Row, ordinal: Int) {
115-
appendValue(row.getFloat(ordinal))
116-
}
117-
}
100+
private[sql] class ByteColumnBuilder extends NativeColumnBuilder(new ByteColumnStats, BYTE)
118101

119-
private[sql] class StringColumnBuilder extends NativeColumnBuilder(STRING) {
120-
override def doAppendFrom(row: Row, ordinal: Int) {
121-
appendValue(row.getString(ordinal))
122-
}
123-
}
102+
private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleColumnStats, DOUBLE)
124103

125-
private[sql] class BinaryColumnBuilder
126-
extends BasicColumnBuilder[BinaryType.type, Array[Byte]]
127-
with NullableColumnBuilder {
104+
private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT)
128105

129-
def columnType = BINARY
106+
private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING)
130107

131-
override def doAppendFrom(row: Row, ordinal: Int) {
132-
appendValue(row(ordinal).asInstanceOf[Array[Byte]])
133-
}
134-
}
108+
private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY)
135109

136110
// TODO (lian) Add support for array, struct and map
137-
private[sql] class GenericColumnBuilder
138-
extends BasicColumnBuilder[DataType, Array[Byte]]
139-
with NullableColumnBuilder {
140-
141-
def columnType = GENERIC
142-
143-
override def doAppendFrom(row: Row, ordinal: Int) {
144-
val serialized = SparkSqlSerializer.serialize(row(ordinal))
145-
buffer = ColumnBuilder.ensureFreeSpace(buffer, columnType.actualSize(serialized))
146-
columnType.append(serialized, buffer)
147-
}
148-
}
111+
private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC)
149112

150113
private[sql] object ColumnBuilder {
151114
val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104

0 commit comments

Comments
 (0)