From 85cc59b15c826b39e6eb5150768fe691a256a1ab Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 27 Mar 2014 17:23:11 +0800 Subject: [PATCH 1/7] Refactored ColumnAccessors & ColumnBuilders to remove duplicate code Primitive setters/getters for (Mutable)Rows are moved to ColumnTypes. --- .../spark/sql/columnar/ColumnAccessor.scala | 96 ++++------------ .../spark/sql/columnar/ColumnBuilder.scala | 105 +++++------------- .../spark/sql/columnar/ColumnType.scala | 79 ++++++++++++- .../NullableColumnAccessorSuite.scala | 3 +- 4 files changed, 128 insertions(+), 155 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index e0c98ecdf8f22..26d5667351fd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -21,7 +21,6 @@ import java.nio.{ByteOrder, ByteBuffer} import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType} import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.execution.SparkSqlSerializer /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is @@ -41,116 +40,61 @@ private[sql] trait ColumnAccessor { protected def underlyingBuffer: ByteBuffer } -private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](buffer: ByteBuffer) +private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( + buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) extends ColumnAccessor { protected def initialize() {} - def columnType: ColumnType[T, JvmType] - def hasNext = buffer.hasRemaining def extractTo(row: MutableRow, ordinal: Int) { - doExtractTo(row, ordinal) + columnType.setField(row, ordinal, columnType.extract(buffer)) } - protected def doExtractTo(row: MutableRow, ordinal: Int) - protected def underlyingBuffer = buffer } private[sql] abstract class NativeColumnAccessor[T <: NativeType]( buffer: ByteBuffer, val columnType: NativeColumnType[T]) - extends BasicColumnAccessor[T, T#JvmType](buffer) - with NullableColumnAccessor - -private[sql] class BooleanColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, BOOLEAN) { + extends BasicColumnAccessor[T, T#JvmType](buffer, columnType) + with NullableColumnAccessor { - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setBoolean(ordinal, columnType.extract(buffer)) - } + type JvmType = T#JvmType } -private[sql] class IntColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, INT) { +private[sql] class BooleanColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, BOOLEAN) - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setInt(ordinal, columnType.extract(buffer)) - } -} +private[sql] class IntColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, INT) private[sql] class ShortColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, SHORT) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setShort(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, SHORT) private[sql] class LongColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, LONG) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setLong(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, LONG) private[sql] class ByteColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, BYTE) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setByte(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, BYTE) private[sql] class DoubleColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, DOUBLE) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setDouble(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, DOUBLE) private[sql] class FloatColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, FLOAT) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setFloat(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, FLOAT) private[sql] class StringColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, STRING) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setString(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, STRING) private[sql] class BinaryColumnAccessor(buffer: ByteBuffer) - extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer) - with NullableColumnAccessor { - - def columnType = BINARY - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row(ordinal) = columnType.extract(buffer) - } -} + extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY) + with NullableColumnAccessor private[sql] class GenericColumnAccessor(buffer: ByteBuffer) - extends BasicColumnAccessor[DataType, Array[Byte]](buffer) - with NullableColumnAccessor { - - def columnType = GENERIC - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - val serialized = columnType.extract(buffer) - row(ordinal) = SparkSqlSerializer.deserialize[Any](serialized) - } -} + extends BasicColumnAccessor[DataType, Array[Byte]](buffer, GENERIC) + with NullableColumnAccessor private[sql] object ColumnAccessor { def apply(b: ByteBuffer): ColumnAccessor = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 3e622adfd3d6a..9394dfa39f2f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -22,7 +22,6 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnBuilder._ -import org.apache.spark.sql.execution.SparkSqlSerializer private[sql] trait ColumnBuilder { /** @@ -30,17 +29,20 @@ private[sql] trait ColumnBuilder { */ def initialize(initialSize: Int, columnName: String = "") + def gatherStats(row: Row, ordinal: Int) {} + def appendFrom(row: Row, ordinal: Int) def build(): ByteBuffer } -private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder { +private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType]( + val columnType: ColumnType[T, JvmType]) + extends ColumnBuilder { private var columnName: String = _ - protected var buffer: ByteBuffer = _ - def columnType: ColumnType[T, JvmType] + protected var buffer: ByteBuffer = _ override def initialize(initialSize: Int, columnName: String = "") = { val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize @@ -49,18 +51,10 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends C buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId) } - // Have to give a concrete implementation to make mixin possible override def appendFrom(row: Row, ordinal: Int) { - doAppendFrom(row, ordinal) - } - - // Concrete `ColumnBuilder`s can override this method to append values - protected def doAppendFrom(row: Row, ordinal: Int) - - // Helper method to append primitive values (to avoid boxing cost) - protected def appendValue(v: JvmType) { - buffer = ensureFreeSpace(buffer, columnType.actualSize(v)) - columnType.append(v, buffer) + val field = columnType.getField(row, ordinal) + buffer = ensureFreeSpace(buffer, columnType.actualSize(field)) + columnType.append(field, buffer) } override def build() = { @@ -70,82 +64,41 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends C } private[sql] abstract class NativeColumnBuilder[T <: NativeType]( - val columnType: NativeColumnType[T]) - extends BasicColumnBuilder[T, T#JvmType] - with NullableColumnBuilder + protected val columnStats: ColumnStats[T], + columnType: NativeColumnType[T]) + extends BasicColumnBuilder[T, T#JvmType](columnType) + with NullableColumnBuilder { -private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(BOOLEAN) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getBoolean(ordinal)) + override def gatherStats(row: Row, ordinal: Int) { + columnStats.gatherStats(row, ordinal) } } -private[sql] class IntColumnBuilder extends NativeColumnBuilder(INT) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getInt(ordinal)) - } -} +private[sql] class BooleanColumnBuilder + extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) -private[sql] class ShortColumnBuilder extends NativeColumnBuilder(SHORT) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getShort(ordinal)) - } -} +private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT) -private[sql] class LongColumnBuilder extends NativeColumnBuilder(LONG) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getLong(ordinal)) - } -} +private[sql] class ShortColumnBuilder extends NativeColumnBuilder(new ShortColumnStats, SHORT) -private[sql] class ByteColumnBuilder extends NativeColumnBuilder(BYTE) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getByte(ordinal)) - } -} +private[sql] class LongColumnBuilder extends NativeColumnBuilder(new LongColumnStats, LONG) -private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(DOUBLE) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getDouble(ordinal)) - } -} +private[sql] class ByteColumnBuilder extends NativeColumnBuilder(new ByteColumnStats, BYTE) -private[sql] class FloatColumnBuilder extends NativeColumnBuilder(FLOAT) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getFloat(ordinal)) - } -} +private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleColumnStats, DOUBLE) -private[sql] class StringColumnBuilder extends NativeColumnBuilder(STRING) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getString(ordinal)) - } -} +private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT) -private[sql] class BinaryColumnBuilder - extends BasicColumnBuilder[BinaryType.type, Array[Byte]] - with NullableColumnBuilder { - - def columnType = BINARY +private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStates, STRING) - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row(ordinal).asInstanceOf[Array[Byte]]) - } -} +private[sql] class BinaryColumnBuilder + extends BasicColumnBuilder[BinaryType.type, Array[Byte]](BINARY) + with NullableColumnBuilder // TODO (lian) Add support for array, struct and map private[sql] class GenericColumnBuilder - extends BasicColumnBuilder[DataType, Array[Byte]] - with NullableColumnBuilder { - - def columnType = GENERIC - - override def doAppendFrom(row: Row, ordinal: Int) { - val serialized = SparkSqlSerializer.serialize(row(ordinal)) - buffer = ColumnBuilder.ensureFreeSpace(buffer, columnType.actualSize(serialized)) - columnType.append(serialized, buffer) - } -} + extends BasicColumnBuilder[DataType, Array[Byte]](GENERIC) + with NullableColumnBuilder private[sql] object ColumnBuilder { val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index a452b86f0cda3..6e85f88726b89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -19,7 +19,10 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkSqlSerializer /** * An abstract class that represents type of a column. Used to append/extract Java objects into/from @@ -50,6 +53,18 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( */ def actualSize(v: JvmType): Int = defaultSize + /** + * Returns `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs + * whenever possible. + */ + def getField(row: Row, ordinal: Int): JvmType + + /** + * Sets `row(ordinal)` to `field`. Subclasses should override this method to avoid boxing/unboxing + * costs whenever possible. + */ + def setField(row: MutableRow, ordinal: Int, value: JvmType) + /** * Creates a duplicated copy of the value. */ @@ -76,6 +91,12 @@ private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { def extract(buffer: ByteBuffer) = { buffer.getInt() } + + override def setField(row: MutableRow, ordinal: Int, value: Int) { + row.setInt(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getInt(ordinal) } private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { @@ -86,6 +107,12 @@ private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { override def extract(buffer: ByteBuffer) = { buffer.getLong() } + + override def setField(row: MutableRow, ordinal: Int, value: Long) { + row.setLong(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getLong(ordinal) } private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { @@ -96,6 +123,12 @@ private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { override def extract(buffer: ByteBuffer) = { buffer.getFloat() } + + override def setField(row: MutableRow, ordinal: Int, value: Float) { + row.setFloat(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getFloat(ordinal) } private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { @@ -106,6 +139,12 @@ private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { override def extract(buffer: ByteBuffer) = { buffer.getDouble() } + + override def setField(row: MutableRow, ordinal: Int, value: Double) { + row.setDouble(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getDouble(ordinal) } private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { @@ -116,6 +155,12 @@ private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { override def extract(buffer: ByteBuffer) = { if (buffer.get() == 1) true else false } + + override def setField(row: MutableRow, ordinal: Int, value: Boolean) { + row.setBoolean(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getBoolean(ordinal) } private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { @@ -126,6 +171,12 @@ private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { override def extract(buffer: ByteBuffer) = { buffer.get() } + + override def setField(row: MutableRow, ordinal: Int, value: Byte) { + row.setByte(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getByte(ordinal) } private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { @@ -136,6 +187,12 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { override def extract(buffer: ByteBuffer) = { buffer.getShort() } + + override def setField(row: MutableRow, ordinal: Int, value: Short) { + row.setShort(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getShort(ordinal) } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { @@ -152,6 +209,12 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { buffer.get(stringBytes, 0, length) new String(stringBytes) } + + override def setField(row: MutableRow, ordinal: Int, value: String) { + row.setString(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getString(ordinal) } private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( @@ -173,12 +236,24 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + row(ordinal) = value + } + + override def getField(row: Row, ordinal: Int) = row(ordinal).asInstanceOf[Array[Byte]] +} // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) +private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + row(ordinal) = SparkSqlSerializer.deserialize[Any](value) + } + + override def getField(row: Row, ordinal: Int) = SparkSqlSerializer.serialize(row(ordinal)) +} private[sql] object ColumnType { implicit def dataTypeToColumnType(dataType: DataType): ColumnType[_, _] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index d413d483f4e7e..0b687e74ed660 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.DataType + import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.DataType class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestData._ From 211331c3016e9357ea0caa884be16fb4a87f3d22 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 28 Mar 2014 01:24:04 +0800 Subject: [PATCH 2/7] WIP: in-memory columnar compression support --- .../spark/sql/columnar/ColumnAccessor.scala | 17 +- .../spark/sql/columnar/ColumnBuilder.scala | 20 +- .../spark/sql/columnar/ColumnStats.scala | 196 ++++++++++++++++++ .../columnar/CompressedColumnAccessor.scala | 41 ++++ .../columnar/CompressedColumnBuilder.scala | 62 ++++++ .../sql/columnar/CompressionAlgorithm.scala | 86 ++++++++ .../sql/columnar/NullableColumnAccessor.scala | 2 +- .../sql/columnar/NullableColumnBuilder.scala | 2 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 38 ++++ .../NullableColumnAccessorSuite.scala | 24 ++- .../columnar/NullableColumnBuilderSuite.scala | 24 ++- 11 files changed, 487 insertions(+), 25 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 26d5667351fd9..4e5cd44acc9f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -41,7 +41,8 @@ private[sql] trait ColumnAccessor { } private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( - buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) + protected val buffer: ByteBuffer, + protected val columnType: ColumnType[T, JvmType]) extends ColumnAccessor { protected def initialize() {} @@ -49,20 +50,20 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( def hasNext = buffer.hasRemaining def extractTo(row: MutableRow, ordinal: Int) { - columnType.setField(row, ordinal, columnType.extract(buffer)) + columnType.setField(row, ordinal, extractSingle(buffer)) } + def extractSingle(buffer: ByteBuffer) = columnType.extract(buffer) + protected def underlyingBuffer = buffer } private[sql] abstract class NativeColumnAccessor[T <: NativeType]( buffer: ByteBuffer, - val columnType: NativeColumnType[T]) - extends BasicColumnAccessor[T, T#JvmType](buffer, columnType) - with NullableColumnAccessor { - - type JvmType = T#JvmType -} + columnType: NativeColumnType[T]) + extends BasicColumnAccessor(buffer, columnType) + with NullableColumnAccessor + with CompressedColumnAccessor[T] private[sql] class BooleanColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, BOOLEAN) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 9394dfa39f2f9..0c10a2b87308f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -29,10 +29,19 @@ private[sql] trait ColumnBuilder { */ def initialize(initialSize: Int, columnName: String = "") + /** + * Gathers statistics information from `row(ordinal)`. + */ def gatherStats(row: Row, ordinal: Int) {} + /** + * Appends `row(ordinal)` to the column builder. + */ def appendFrom(row: Row, ordinal: Int) + /** + * Returns the final columnar byte buffer. + */ def build(): ByteBuffer } @@ -40,14 +49,16 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType]( val columnType: ColumnType[T, JvmType]) extends ColumnBuilder { - private var columnName: String = _ + protected var columnName: String = _ protected var buffer: ByteBuffer = _ override def initialize(initialSize: Int, columnName: String = "") = { val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize this.columnName = columnName - buffer = ByteBuffer.allocate(4 + 4 + size * columnType.defaultSize) + + // Reserves 4 bytes for column type ID + buffer = ByteBuffer.allocate(4 + size * columnType.defaultSize) buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId) } @@ -66,8 +77,9 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType]( private[sql] abstract class NativeColumnBuilder[T <: NativeType]( protected val columnStats: ColumnStats[T], columnType: NativeColumnType[T]) - extends BasicColumnBuilder[T, T#JvmType](columnType) - with NullableColumnBuilder { + extends BasicColumnBuilder(columnType) + with NullableColumnBuilder + with CompressedColumnBuilder[T] { override def gatherStats(row: Row, ordinal: Int) { columnStats.gatherStats(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala new file mode 100644 index 0000000000000..2fefd1a75fb72 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.types._ + +private[sql] sealed abstract class ColumnStats[T <: NativeType] extends Serializable{ + type JvmType = T#JvmType + + protected var (_lower, _upper) = initialBounds + + protected val ordering: Ordering[JvmType] + + protected def columnType: NativeColumnType[T] + + /** + * Closed lower bound of this column. + */ + def lowerBound = _lower + + /** + * Closed upper bound of this column. + */ + def upperBound = _upper + + /** + * Initial values for the closed lower/upper bounds, in the format of `(lower, upper)`. + */ + protected def initialBounds: (JvmType, JvmType) + + /** + * Gathers statistics information from `row(ordinal)`. + */ + @inline def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (ordering.gt(field, upperBound)) _upper = field + if (ordering.lt(field, lowerBound)) _lower = field + } + + /** + * Returns `true` if `lower <= row(ordinal) <= upper`. + */ + @inline def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + ordering.lteq(lowerBound, field) && ordering.lteq(field, upperBound) + } + + /** + * Returns `true` if `row(ordinal) < upper` holds. + */ + @inline def isAbove(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + ordering.lt(field, upperBound) + } + + /** + * Returns `true` if `lower < row(ordinal)` holds. + */ + @inline def isBelow(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + ordering.lt(lowerBound, field) + } + + /** + * Returns `true` if `row(ordinal) <= upper` holds. + */ + @inline def isAtOrAbove(row: Row, ordinal: Int) = { + contains(row, ordinal) || isAbove(row, ordinal) + } + + /** + * Returns `true` if `lower <= row(ordinal)` holds. + */ + @inline def isAtOrBelow(row: Row, ordinal: Int) = { + contains(row, ordinal) || isBelow(row, ordinal) + } +} + +private[sql] abstract class BasicColumnStats[T <: NativeType]( + protected val columnType: NativeColumnType[T]) + extends ColumnStats[T] + +private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (true, false) +} + +private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (Byte.MaxValue, Byte.MinValue) +} + +private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (Short.MaxValue, Short.MinValue) +} + +private[sql] class LongColumnStats extends BasicColumnStats(LONG) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (Long.MaxValue, Long.MinValue) +} + +private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (Double.MaxValue, Double.MinValue) +} + +private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (Float.MaxValue, Float.MinValue) +} + +private[sql] class IntColumnStats extends BasicColumnStats(INT) { + private object OrderedState extends Enumeration { + val Uninitialized, Initialized, Ascending, Descending, Unordered = Value + } + + import OrderedState._ + + private var orderedState = Uninitialized + private var lastValue: Int = _ + private var _maxDelta: Int = _ + + def isAscending = orderedState != Descending && orderedState != Unordered + def isDescending = orderedState != Ascending && orderedState != Unordered + def isOrdered = isAscending || isDescending + def maxDelta = _maxDelta + + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (Int.MaxValue, Int.MinValue) + + override def gatherStats(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + + orderedState = orderedState match { + case Uninitialized => + lastValue = field + Initialized + + case Initialized => + // If all the integers in the column are the same, ordered state is set to Ascending. + // TODO (lian) Confirm whether this is the standard behaviour. + val nextState = if (field >= lastValue) Ascending else Descending + _maxDelta = math.abs(field - lastValue) + lastValue = field + nextState + + case Ascending if field < lastValue => + Unordered + + case Descending if field > lastValue => + Unordered + + case state @ (Ascending | Descending) => + _maxDelta = _maxDelta.max(field - lastValue) + lastValue = field + state + } + } +} + +private[sql] class StringColumnStates extends BasicColumnStats(STRING) { + override protected val ordering = implicitly[Ordering[JvmType]] + override protected def initialBounds = (null, null) + + override def contains(row: Row, ordinal: Int) = { + !(upperBound eq null) && super.contains(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + !(upperBound eq null) && super.isAbove(row, ordinal) + } + + override def isBelow(row: Row, ordinal: Int) = { + !(lowerBound eq null) && super.isBelow(row, ordinal) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala new file mode 100644 index 0000000000000..b7c52de8a60d1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar.CompressionAlgorithm.NoopDecoder +import org.apache.spark.sql.columnar.CompressionType._ + +private[sql] trait CompressedColumnAccessor[T <: NativeType] extends ColumnAccessor { + this: BasicColumnAccessor[T, T#JvmType] => + + private var decoder: Iterator[T#JvmType] = _ + + abstract override protected def initialize() = { + super.initialize() + + decoder = underlyingBuffer.getInt() match { + case id if id == Noop.id => new NoopDecoder[T](buffer, columnType) + case _ => throw new UnsupportedOperationException() + } + } + + abstract override def extractSingle(buffer: ByteBuffer) = decoder.next() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala new file mode 100644 index 0000000000000..fa31e48f35cd7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.sql.catalyst.types.NativeType + +private[sql] trait CompressedColumnBuilder[T <: NativeType] extends ColumnBuilder with Logging { + this: BasicColumnBuilder[T, T#JvmType] => + + val compressionSchemes = Seq(new CompressionAlgorithm.Noop) + .filter(_.supports(columnType)) + + def isWorthCompressing(scheme: CompressionAlgorithm) = { + scheme.compressionRatio < 0.8 + } + + abstract override def gatherStats(row: Row, ordinal: Int) { + compressionSchemes.foreach { + val field = columnType.getField(row, ordinal) + _.gatherCompressibilityStats(field, columnType) + } + + super.gatherStats(row, ordinal) + } + + abstract override def build() = { + val rawBuffer = super.build() + + if (compressionSchemes.isEmpty) { + logger.info(s"Compression scheme chosen for [$columnName] is ${CompressionType.Noop}") + new CompressionAlgorithm.Noop().compress(rawBuffer, columnType) + } else { + val candidateScheme = compressionSchemes.minBy(_.compressionRatio) + + logger.info( + s"Compression scheme chosen for [$columnName] is ${candidateScheme.compressionType} " + + s"ration ${candidateScheme.compressionRatio}") + + if (isWorthCompressing(candidateScheme)) { + candidateScheme.compress(rawBuffer, columnType) + } else { + new CompressionAlgorithm.Noop().compress(rawBuffer, columnType) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala new file mode 100644 index 0000000000000..855767f69b4e0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala @@ -0,0 +1,86 @@ +package org.apache.spark.sql.columnar + +import java.nio.{ByteOrder, ByteBuffer} + +import org.apache.spark.sql.catalyst.types.NativeType + +private[sql] object CompressionType extends Enumeration { + type CompressionType = Value + + val Default, Noop, RLE, Dictionary, BooleanBitSet, IntDelta, LongDelta = Value +} + +private[sql] trait CompressionAlgorithm { + def compressionType: CompressionType.Value + + def supports(columnType: ColumnType[_, _]): Boolean + + def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) {} + + def compressedSize: Int + + def uncompressedSize: Int + + def compressionRatio: Double = compressedSize.toDouble / uncompressedSize + + def compress[T <: NativeType](from: ByteBuffer, columnType: ColumnType[T, T#JvmType]): ByteBuffer +} + +private[sql] object CompressionAlgorithm { + def apply(typeId: Int) = typeId match { + case CompressionType.Noop => new CompressionAlgorithm.Noop + case _ => throw new UnsupportedOperationException() + } + + class Noop extends CompressionAlgorithm { + override def uncompressedSize = 0 + override def compressedSize = 0 + override def compressionRatio = 1.0 + override def supports(columnType: ColumnType[_, _]) = true + override def compressionType = CompressionType.Noop + + override def compress[T <: NativeType]( + from: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + // Reserves 4 bytes for compression type + val to = ByteBuffer.allocate(from.limit + 4).order(ByteOrder.nativeOrder) + copyHeader(from, to) + + // Writes compression type ID and copies raw contents + to.putInt(CompressionType.Noop.id).put(from).rewind() + to + } + } + + class NoopDecoder[T <: NativeType](buffer: ByteBuffer, columnType: ColumnType[T, T#JvmType]) + extends Iterator[T#JvmType] { + + override def next() = columnType.extract(buffer) + + override def hasNext = buffer.hasRemaining + } + + def copyNullInfo(from: ByteBuffer, to: ByteBuffer) { + // Writes null count + val nullCount = from.getInt() + to.putInt(nullCount) + + // Writes null positions + var i = 0 + while (i < nullCount) { + to.putInt(from.getInt()) + i += 1 + } + } + + def copyHeader(from: ByteBuffer, to: ByteBuffer) { + // Writes column type ID + to.putInt(from.getInt()) + + // Copies null count and null positions + copyNullInfo(from, to) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index 2970c609b928d..7d49ab07f7a53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -29,7 +29,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { private var nextNullIndex: Int = _ private var pos: Int = 0 - abstract override def initialize() { + abstract override protected def initialize() { nullsBuffer = underlyingBuffer.duplicate().order(ByteOrder.nativeOrder()) nullCount = nullsBuffer.getInt() nextNullIndex = if (nullCount > 0) nullsBuffer.getInt() else -1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 048d1f05c7df2..8712fdb283659 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -71,7 +71,7 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { // | ... | Non-null part (without column type ID) // +---------+ val buffer = ByteBuffer - .allocate(4 + nullDataLen + nonNulls.limit) + .allocate(4 + 4 + nullDataLen + nonNulls.remaining()) .order(ByteOrder.nativeOrder()) .putInt(typeId) .putInt(nullCount) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala new file mode 100644 index 0000000000000..0fa5323ba2060 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +class ColumnStatsSuite extends FunSuite { + test("Boolean") { + val stats = new BooleanColumnStats + val row = new GenericMutableRow(1) + + row(0) = false + stats.gatherStats(row, 0) + assert(stats.lowerBound === false) + assert(stats.upperBound === false) + + row(0) = true + stats.gatherStats(row, 0) + assert(stats.lowerBound === false) + assert(stats.upperBound === true) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 0b687e74ed660..61f7338913791 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -17,11 +17,27 @@ package org.apache.spark.sql.columnar +import java.nio.ByteBuffer + import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.DataType +class TestNullableColumnAccessor[T <: DataType, JvmType]( + buffer: ByteBuffer, + columnType: ColumnType[T, JvmType]) + extends BasicColumnAccessor(buffer, columnType) + with NullableColumnAccessor + +object TestNullableColumnAccessor { + def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) = { + // Skips the column type ID + buffer.getInt() + new TestNullableColumnAccessor(buffer, columnType) + } +} + class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestData._ @@ -33,20 +49,20 @@ class NullableColumnAccessorSuite extends FunSuite { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") test(s"$typeName accessor: empty column") { - val builder = ColumnBuilder(columnType.typeId, 4) - val accessor = ColumnAccessor(builder.build()) + val builder = TestNullableColumnBuilder(columnType) + val accessor = TestNullableColumnAccessor(builder.build(), columnType) assert(!accessor.hasNext) } test(s"$typeName accessor: access null values") { - val builder = ColumnBuilder(columnType.typeId, 4) + val builder = TestNullableColumnBuilder(columnType) (0 until 4).foreach { _ => builder.appendFrom(nonNullRandomRow, columnType.typeId) builder.appendFrom(nullRow, columnType.typeId) } - val accessor = ColumnAccessor(builder.build()) + val accessor = TestNullableColumnAccessor(builder.build(), columnType) val row = new GenericMutableRow(1) (0 until 4).foreach { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 5222a47e1ab87..3732aaaba6687 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -19,9 +19,21 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer +class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) + extends BasicColumnBuilder(columnType) + with NullableColumnBuilder + +object TestNullableColumnBuilder { + def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) = { + val builder = new TestNullableColumnBuilder(columnType) + builder.initialize(initialSize) + builder + } +} + class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestData._ @@ -30,23 +42,21 @@ class NullableColumnBuilderSuite extends FunSuite { } def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { - val columnBuilder = ColumnBuilder(columnType.typeId) val typeName = columnType.getClass.getSimpleName.stripSuffix("$") test(s"$typeName column builder: empty column") { - columnBuilder.initialize(4) - + val columnBuilder = TestNullableColumnBuilder(columnType) val buffer = columnBuilder.build() // For column type ID assert(buffer.getInt() === columnType.typeId) // For null count - assert(buffer.getInt === 0) + assert(buffer.getInt() === 0) assert(!buffer.hasRemaining) } test(s"$typeName column builder: buffer size auto growth") { - columnBuilder.initialize(4) + val columnBuilder = TestNullableColumnBuilder(columnType) (0 until 4) foreach { _ => columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) @@ -61,7 +71,7 @@ class NullableColumnBuilderSuite extends FunSuite { } test(s"$typeName column builder: null values") { - columnBuilder.initialize(4) + val columnBuilder = TestNullableColumnBuilder(columnType) (0 until 4) foreach { _ => columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) From 2780d6acad290277e0240599b6fe9c40a84b429a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 1 Apr 2014 01:48:09 +0800 Subject: [PATCH 3/7] [WIP] in-memory columnar compression support * Added two more compression schemes (RLE & dictionary encoding) * Moved compression support code to columnar.compression * Various refactoring --- .../spark/sql/columnar/ColumnAccessor.scala | 14 +- .../spark/sql/columnar/ColumnBuilder.scala | 44 ++- .../spark/sql/columnar/ColumnStats.scala | 182 ++++++----- .../spark/sql/columnar/ColumnType.scala | 4 +- .../columnar/CompressedColumnBuilder.scala | 62 ---- .../sql/columnar/CompressionAlgorithm.scala | 86 ------ .../sql/columnar/NullableColumnBuilder.scala | 27 +- .../CompressibleColumnAccessor.scala} | 19 +- .../CompressibleColumnBuilder.scala | 95 ++++++ .../compression/CompressionScheme.scala | 92 ++++++ .../compression/compressionSchemes.scala | 283 ++++++++++++++++++ .../columnar/inMemoryColumnarOperators.scala | 4 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 74 ++++- .../sql/columnar/ColumnarQuerySuite.scala | 4 +- .../columnar/NullableColumnBuilderSuite.scala | 2 +- 15 files changed, 698 insertions(+), 294 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala rename sql/core/src/main/scala/org/apache/spark/sql/columnar/{CompressedColumnAccessor.scala => compression/CompressibleColumnAccessor.scala} (61%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 4e5cd44acc9f3..ffd4894b5213d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -21,6 +21,7 @@ import java.nio.{ByteOrder, ByteBuffer} import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType} import org.apache.spark.sql.catalyst.expressions.MutableRow +import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is @@ -53,17 +54,17 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( columnType.setField(row, ordinal, extractSingle(buffer)) } - def extractSingle(buffer: ByteBuffer) = columnType.extract(buffer) + def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer) protected def underlyingBuffer = buffer } private[sql] abstract class NativeColumnAccessor[T <: NativeType]( - buffer: ByteBuffer, - columnType: NativeColumnType[T]) + override protected val buffer: ByteBuffer, + override protected val columnType: NativeColumnType[T]) extends BasicColumnAccessor(buffer, columnType) with NullableColumnAccessor - with CompressedColumnAccessor[T] + with CompressibleColumnAccessor[T] private[sql] class BooleanColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, BOOLEAN) @@ -98,9 +99,8 @@ private[sql] class GenericColumnAccessor(buffer: ByteBuffer) with NullableColumnAccessor private[sql] object ColumnAccessor { - def apply(b: ByteBuffer): ColumnAccessor = { - // The first 4 bytes in the buffer indicates the column type. - val buffer = b.duplicate().order(ByteOrder.nativeOrder()) + def apply(buffer: ByteBuffer): ColumnAccessor = { + // The first 4 bytes in the buffer indicate the column type. val columnTypeId = buffer.getInt() columnTypeId match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 0c10a2b87308f..048ee66bff44b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -22,6 +22,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnBuilder._ +import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder} private[sql] trait ColumnBuilder { /** @@ -30,14 +31,14 @@ private[sql] trait ColumnBuilder { def initialize(initialSize: Int, columnName: String = "") /** - * Gathers statistics information from `row(ordinal)`. + * Appends `row(ordinal)` to the column builder. */ - def gatherStats(row: Row, ordinal: Int) {} + def appendFrom(row: Row, ordinal: Int) /** - * Appends `row(ordinal)` to the column builder. + * Column statistics information */ - def appendFrom(row: Row, ordinal: Int) + def columnStats: ColumnStats[_, _] /** * Returns the final columnar byte buffer. @@ -45,7 +46,8 @@ private[sql] trait ColumnBuilder { def build(): ByteBuffer } -private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType]( +private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( + val columnStats: ColumnStats[T, JvmType], val columnType: ColumnType[T, JvmType]) extends ColumnBuilder { @@ -74,20 +76,20 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType]( } } -private[sql] abstract class NativeColumnBuilder[T <: NativeType]( - protected val columnStats: ColumnStats[T], - columnType: NativeColumnType[T]) - extends BasicColumnBuilder(columnType) +private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType]) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) with NullableColumnBuilder - with CompressedColumnBuilder[T] { - override def gatherStats(row: Row, ordinal: Int) { - columnStats.gatherStats(row, ordinal) - } -} +private[sql] abstract class NativeColumnBuilder[T <: NativeType]( + override val columnStats: NativeColumnStats[T], + override val columnType: NativeColumnType[T]) + extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType) + with NullableColumnBuilder + with AllCompressionSchemes + with CompressibleColumnBuilder[T] -private[sql] class BooleanColumnBuilder - extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) +private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT) @@ -101,16 +103,12 @@ private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleCol private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT) -private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStates, STRING) +private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) -private[sql] class BinaryColumnBuilder - extends BasicColumnBuilder[BinaryType.type, Array[Byte]](BINARY) - with NullableColumnBuilder +private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY) // TODO (lian) Add support for array, struct and map -private[sql] class GenericColumnBuilder - extends BasicColumnBuilder[DataType, Array[Byte]](GENERIC) - with NullableColumnBuilder +private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC) private[sql] object ColumnBuilder { val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 2fefd1a75fb72..43096ee0da945 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -20,130 +20,169 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ -private[sql] sealed abstract class ColumnStats[T <: NativeType] extends Serializable{ - type JvmType = T#JvmType +private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable{ + /** + * Closed lower bound of this column. + */ + def lowerBound: JvmType - protected var (_lower, _upper) = initialBounds + /** + * Closed upper bound of this column. + */ + def upperBound: JvmType - protected val ordering: Ordering[JvmType] + /** + * Gathers statistics information from `row(ordinal)`. + */ + def gatherStats(row: Row, ordinal: Int) - protected def columnType: NativeColumnType[T] + /** + * Returns `true` if `lower <= row(ordinal) <= upper`. + */ + def contains(row: Row, ordinal: Int): Boolean /** - * Closed lower bound of this column. + * Returns `true` if `row(ordinal) < upper` holds. */ - def lowerBound = _lower + def isAbove(row: Row, ordinal: Int): Boolean /** - * Closed upper bound of this column. + * Returns `true` if `lower < row(ordinal)` holds. */ - def upperBound = _upper + def isBelow(row: Row, ordinal: Int): Boolean /** - * Initial values for the closed lower/upper bounds, in the format of `(lower, upper)`. + * Returns `true` if `row(ordinal) <= upper` holds. */ - protected def initialBounds: (JvmType, JvmType) + def isAtOrAbove(row: Row, ordinal: Int): Boolean /** - * Gathers statistics information from `row(ordinal)`. + * Returns `true` if `lower <= row(ordinal)` holds. */ - @inline def gatherStats(row: Row, ordinal: Int) { + def isAtOrBelow(row: Row, ordinal: Int): Boolean +} + +private[sql] sealed abstract class NativeColumnStats[T <: NativeType] + extends ColumnStats[T, T#JvmType] { + + type JvmType = T#JvmType + + protected var (_lower, _upper) = initialBounds + + val ordering: Ordering[JvmType] + + def initialBounds: (JvmType, JvmType) + + protected def columnType: NativeColumnType[T] + + override def lowerBound = _lower + + override def upperBound = _upper + + override def gatherStats(row: Row, ordinal: Int) { val field = columnType.getField(row, ordinal) - if (ordering.gt(field, upperBound)) _upper = field - if (ordering.lt(field, lowerBound)) _lower = field + if (upperBound == null || ordering.gt(field, upperBound)) _upper = field + if (lowerBound == null || ordering.lt(field, lowerBound)) _lower = field } - /** - * Returns `true` if `lower <= row(ordinal) <= upper`. - */ - @inline def contains(row: Row, ordinal: Int) = { + override def contains(row: Row, ordinal: Int) = { val field = columnType.getField(row, ordinal) ordering.lteq(lowerBound, field) && ordering.lteq(field, upperBound) } - /** - * Returns `true` if `row(ordinal) < upper` holds. - */ - @inline def isAbove(row: Row, ordinal: Int) = { + override def isAbove(row: Row, ordinal: Int) = { val field = columnType.getField(row, ordinal) ordering.lt(field, upperBound) } - /** - * Returns `true` if `lower < row(ordinal)` holds. - */ - @inline def isBelow(row: Row, ordinal: Int) = { + override def isBelow(row: Row, ordinal: Int) = { val field = columnType.getField(row, ordinal) ordering.lt(lowerBound, field) } - /** - * Returns `true` if `row(ordinal) <= upper` holds. - */ - @inline def isAtOrAbove(row: Row, ordinal: Int) = { + override def isAtOrAbove(row: Row, ordinal: Int) = { contains(row, ordinal) || isAbove(row, ordinal) } - /** - * Returns `true` if `lower <= row(ordinal)` holds. - */ - @inline def isAtOrBelow(row: Row, ordinal: Int) = { + override def isAtOrBelow(row: Row, ordinal: Int) = { contains(row, ordinal) || isBelow(row, ordinal) } } +private[sql] class NoopColumnStats[T <: DataType, JvmType] extends ColumnStats[T, JvmType] { + override def isAtOrBelow(row: Row, ordinal: Int) = true + + override def isAtOrAbove(row: Row, ordinal: Int) = true + + override def isBelow(row: Row, ordinal: Int) = true + + override def isAbove(row: Row, ordinal: Int) = true + + override def contains(row: Row, ordinal: Int) = true + + override def gatherStats(row: Row, ordinal: Int) {} + + override def upperBound = null.asInstanceOf[JvmType] + + override def lowerBound = null.asInstanceOf[JvmType] +} + private[sql] abstract class BasicColumnStats[T <: NativeType]( protected val columnType: NativeColumnType[T]) - extends ColumnStats[T] + extends NativeColumnStats[T] private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (true, false) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (true, false) } private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (Byte.MaxValue, Byte.MinValue) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (Byte.MaxValue, Byte.MinValue) } private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (Short.MaxValue, Short.MinValue) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (Short.MaxValue, Short.MinValue) } private[sql] class LongColumnStats extends BasicColumnStats(LONG) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (Long.MaxValue, Long.MinValue) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (Long.MaxValue, Long.MinValue) } private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (Double.MaxValue, Double.MinValue) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (Double.MaxValue, Double.MinValue) } private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (Float.MaxValue, Float.MinValue) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (Float.MaxValue, Float.MinValue) } -private[sql] class IntColumnStats extends BasicColumnStats(INT) { - private object OrderedState extends Enumeration { - val Uninitialized, Initialized, Ascending, Descending, Unordered = Value - } +object IntColumnStats { + val UNINITIALIZED = 0 + val INITIALIZED = 1 + val ASCENDING = 2 + val DESCENDING = 3 + val UNORDERED = 4 +} - import OrderedState._ +private[sql] class IntColumnStats extends BasicColumnStats(INT) { + import IntColumnStats._ - private var orderedState = Uninitialized + private var orderedState = UNINITIALIZED private var lastValue: Int = _ private var _maxDelta: Int = _ - def isAscending = orderedState != Descending && orderedState != Unordered - def isDescending = orderedState != Ascending && orderedState != Unordered + def isAscending = orderedState != DESCENDING && orderedState != UNORDERED + def isDescending = orderedState != ASCENDING && orderedState != UNORDERED def isOrdered = isAscending || isDescending def maxDelta = _maxDelta - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (Int.MaxValue, Int.MinValue) + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (Int.MaxValue, Int.MinValue) override def gatherStats(row: Row, ordinal: Int) = { val field = columnType.getField(row, ordinal) @@ -152,35 +191,38 @@ private[sql] class IntColumnStats extends BasicColumnStats(INT) { if (field < lowerBound) _lower = field orderedState = orderedState match { - case Uninitialized => + case UNINITIALIZED => lastValue = field - Initialized + INITIALIZED - case Initialized => + case INITIALIZED => // If all the integers in the column are the same, ordered state is set to Ascending. // TODO (lian) Confirm whether this is the standard behaviour. - val nextState = if (field >= lastValue) Ascending else Descending + val nextState = if (field >= lastValue) ASCENDING else DESCENDING _maxDelta = math.abs(field - lastValue) lastValue = field nextState - case Ascending if field < lastValue => - Unordered + case ASCENDING if field < lastValue => + UNORDERED - case Descending if field > lastValue => - Unordered + case DESCENDING if field > lastValue => + UNORDERED - case state @ (Ascending | Descending) => + case state @ (ASCENDING | DESCENDING) => _maxDelta = _maxDelta.max(field - lastValue) lastValue = field state + + case _ => + orderedState } } } -private[sql] class StringColumnStates extends BasicColumnStats(STRING) { - override protected val ordering = implicitly[Ordering[JvmType]] - override protected def initialBounds = (null, null) +private[sql] class StringColumnStats extends BasicColumnStats(STRING) { + override val ordering = implicitly[Ordering[JvmType]] + override def initialBounds = (null, null) override def contains(row: Row, ordinal: Int) = { !(upperBound eq null) && super.contains(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 6e85f88726b89..a761c42b2aba2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ @@ -80,7 +82,7 @@ private[sql] abstract class NativeColumnType[T <: NativeType]( /** * Scala TypeTag. Can be used to create primitive arrays and hash tables. */ - def scalaTag = dataType.tag + def scalaTag: TypeTag[dataType.JvmType] = dataType.tag } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala deleted file mode 100644 index fa31e48f35cd7..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnBuilder.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.columnar - -import org.apache.spark.sql.{Logging, Row} -import org.apache.spark.sql.catalyst.types.NativeType - -private[sql] trait CompressedColumnBuilder[T <: NativeType] extends ColumnBuilder with Logging { - this: BasicColumnBuilder[T, T#JvmType] => - - val compressionSchemes = Seq(new CompressionAlgorithm.Noop) - .filter(_.supports(columnType)) - - def isWorthCompressing(scheme: CompressionAlgorithm) = { - scheme.compressionRatio < 0.8 - } - - abstract override def gatherStats(row: Row, ordinal: Int) { - compressionSchemes.foreach { - val field = columnType.getField(row, ordinal) - _.gatherCompressibilityStats(field, columnType) - } - - super.gatherStats(row, ordinal) - } - - abstract override def build() = { - val rawBuffer = super.build() - - if (compressionSchemes.isEmpty) { - logger.info(s"Compression scheme chosen for [$columnName] is ${CompressionType.Noop}") - new CompressionAlgorithm.Noop().compress(rawBuffer, columnType) - } else { - val candidateScheme = compressionSchemes.minBy(_.compressionRatio) - - logger.info( - s"Compression scheme chosen for [$columnName] is ${candidateScheme.compressionType} " + - s"ration ${candidateScheme.compressionRatio}") - - if (isWorthCompressing(candidateScheme)) { - candidateScheme.compress(rawBuffer, columnType) - } else { - new CompressionAlgorithm.Noop().compress(rawBuffer, columnType) - } - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala deleted file mode 100644 index 855767f69b4e0..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressionAlgorithm.scala +++ /dev/null @@ -1,86 +0,0 @@ -package org.apache.spark.sql.columnar - -import java.nio.{ByteOrder, ByteBuffer} - -import org.apache.spark.sql.catalyst.types.NativeType - -private[sql] object CompressionType extends Enumeration { - type CompressionType = Value - - val Default, Noop, RLE, Dictionary, BooleanBitSet, IntDelta, LongDelta = Value -} - -private[sql] trait CompressionAlgorithm { - def compressionType: CompressionType.Value - - def supports(columnType: ColumnType[_, _]): Boolean - - def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) {} - - def compressedSize: Int - - def uncompressedSize: Int - - def compressionRatio: Double = compressedSize.toDouble / uncompressedSize - - def compress[T <: NativeType](from: ByteBuffer, columnType: ColumnType[T, T#JvmType]): ByteBuffer -} - -private[sql] object CompressionAlgorithm { - def apply(typeId: Int) = typeId match { - case CompressionType.Noop => new CompressionAlgorithm.Noop - case _ => throw new UnsupportedOperationException() - } - - class Noop extends CompressionAlgorithm { - override def uncompressedSize = 0 - override def compressedSize = 0 - override def compressionRatio = 1.0 - override def supports(columnType: ColumnType[_, _]) = true - override def compressionType = CompressionType.Noop - - override def compress[T <: NativeType]( - from: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - - // Reserves 4 bytes for compression type - val to = ByteBuffer.allocate(from.limit + 4).order(ByteOrder.nativeOrder) - copyHeader(from, to) - - // Writes compression type ID and copies raw contents - to.putInt(CompressionType.Noop.id).put(from).rewind() - to - } - } - - class NoopDecoder[T <: NativeType](buffer: ByteBuffer, columnType: ColumnType[T, T#JvmType]) - extends Iterator[T#JvmType] { - - override def next() = columnType.extract(buffer) - - override def hasNext = buffer.hasRemaining - } - - def copyNullInfo(from: ByteBuffer, to: ByteBuffer) { - // Writes null count - val nullCount = from.getInt() - to.putInt(nullCount) - - // Writes null positions - var i = 0 - while (i < nullCount) { - to.putInt(from.getInt()) - i += 1 - } - } - - def copyHeader(from: ByteBuffer, to: ByteBuffer) { - // Writes column type ID - to.putInt(from.getInt()) - - // Copies null count and null positions - copyNullInfo(from, to) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 8712fdb283659..2a3b6fc1e46d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -22,10 +22,18 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row /** - * Builds a nullable column. The byte buffer of a nullable column contains: - * - 4 bytes for the null count (number of nulls) - * - positions for each null, in ascending order - * - the non-null data (column data type, compression type, data...) + * A stackable trait used for building byte buffer for a column containing null values. Memory + * layout of the final byte buffer is: + * {{{ + * .----------------------- Column type ID (4 bytes) + * | .------------------- Null count N (4 bytes) + * | | .--------------- Null positions (4 x N bytes, empty if null count is zero) + * | | | .--------- Non-null elements + * V V V V + * +---+---+-----+---------+ + * | | | ... | ... ... | + * +---+---+-----+---------+ + * }}} */ private[sql] trait NullableColumnBuilder extends ColumnBuilder { private var nulls: ByteBuffer = _ @@ -59,17 +67,6 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { nulls.limit(nullDataLen) nulls.rewind() - // Column type ID is moved to the front, follows the null count, then non-null data - // - // +---------+ - // | 4 bytes | Column type ID - // +---------+ - // | 4 bytes | Null count - // +---------+ - // | ... | Null positions (if null count is not zero) - // +---------+ - // | ... | Non-null part (without column type ID) - // +---------+ val buffer = ByteBuffer .allocate(4 + 4 + nullDataLen + nonNulls.remaining()) .order(ByteOrder.nativeOrder()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala similarity index 61% rename from sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala rename to sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index b7c52de8a60d1..878cb84de106f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CompressedColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -15,27 +15,22 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar +package org.apache.spark.sql.columnar.compression import java.nio.ByteBuffer import org.apache.spark.sql.catalyst.types.NativeType -import org.apache.spark.sql.columnar.CompressionAlgorithm.NoopDecoder -import org.apache.spark.sql.columnar.CompressionType._ +import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} -private[sql] trait CompressedColumnAccessor[T <: NativeType] extends ColumnAccessor { - this: BasicColumnAccessor[T, T#JvmType] => +private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor { + this: NativeColumnAccessor[T] => - private var decoder: Iterator[T#JvmType] = _ + private var decoder: Decoder[T] = _ abstract override protected def initialize() = { super.initialize() - - decoder = underlyingBuffer.getInt() match { - case id if id == Noop.id => new NoopDecoder[T](buffer, columnType) - case _ => throw new UnsupportedOperationException() - } + decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType) } - abstract override def extractSingle(buffer: ByteBuffer) = decoder.next() + abstract override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala new file mode 100644 index 0000000000000..bb15ca191069d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import java.nio.{ByteBuffer, ByteOrder} + +import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} + +/** + * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of + * the final byte buffer is: + * {{{ + * .--------------------------- 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 + * }}} + */ +private[sql] trait CompressibleColumnBuilder[T <: NativeType] + extends ColumnBuilder with WithCompressionSchemes with Logging { + + this: NativeColumnBuilder[T] => + + import CompressionScheme._ + + val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder) + + private def isWorthCompressing(encoder: Encoder) = { + encoder.compressionRatio < 0.8 + } + + private def gatherCompressibilityStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + + var i = 0 + while (i < compressionEncoders.length) { + compressionEncoders(i).gatherCompressibilityStats(field, columnType) + i += 1 + } + } + + abstract override def appendFrom(row: Row, ordinal: Int) { + super.appendFrom(row, ordinal) + gatherCompressibilityStats(row, ordinal) + } + + abstract override def build() = { + val rawBuffer = super.build() + val encoder = { + val candidate = compressionEncoders.minBy(_.compressionRatio) + if (isWorthCompressing(candidate)) candidate else PassThrough.encoder + } + + val headerSize = columnHeaderSize(rawBuffer) + val compressedSize = if (encoder.compressedSize == 0) { + rawBuffer.limit - headerSize + } else { + encoder.compressedSize + } + + // Reserves 4 bytes for compression scheme ID + val compressedBuffer = ByteBuffer + .allocate(headerSize + 4 + compressedSize) + .order(ByteOrder.nativeOrder) + + copyColumnHeader(rawBuffer, compressedBuffer) + + logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + encoder.compress(rawBuffer, compressedBuffer, columnType) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala new file mode 100644 index 0000000000000..be2f359553020 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} + +private[sql] trait Encoder { + def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) {} + + def compressedSize: Int + + def uncompressedSize: Int + + def compressionRatio: Double = { + if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0 + } + + def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]): ByteBuffer +} + +private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType] + +private[sql] trait CompressionScheme { + def typeId: Int + + def supports(columnType: ColumnType[_, _]): Boolean + + def encoder: Encoder + + def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] +} + +private[sql] trait WithCompressionSchemes { + def schemes: Seq[CompressionScheme] +} + +private[sql] trait AllCompressionSchemes extends WithCompressionSchemes { + override val schemes = Seq(PassThrough, RunLengthEncoding, DictionaryEncoding) +} + +private[sql] object CompressionScheme { + def apply(typeId: Int): CompressionScheme = typeId match { + case PassThrough.typeId => PassThrough + case _ => throw new UnsupportedOperationException() + } + + def copyColumnHeader(from: ByteBuffer, to: ByteBuffer) { + // Writes column type ID + to.putInt(from.getInt()) + + // Writes null count + val nullCount = from.getInt() + to.putInt(nullCount) + + // Writes null positions + var i = 0 + while (i < nullCount) { + to.putInt(from.getInt()) + i += 1 + } + } + + def columnHeaderSize(columnBuffer: ByteBuffer): Int = { + val header = columnBuffer.duplicate() + val nullCount = header.getInt(4) + // Column type ID + null count + null positions + 4 + 4 + 4 * nullCount + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala new file mode 100644 index 0000000000000..f77e15f1036e6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import java.nio.ByteBuffer + +import scala.collection.mutable +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.runtimeMirror + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ + +private[sql] object PassThrough extends CompressionScheme { + override val typeId = 0 + + override def supports(columnType: ColumnType[_, _]) = true + + override def encoder = new this.Encoder + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer, columnType) + } + + class Encoder extends compression.Encoder { + override def uncompressedSize = 0 + + override def compressedSize = 0 + + override def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + // Writes compression type ID and copies raw contents + to.putInt(PassThrough.typeId).put(from).rewind() + to + } + } + + class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + extends compression.Decoder[T] { + + override def next() = columnType.extract(buffer) + + override def hasNext = buffer.hasRemaining + } +} + +private[sql] object RunLengthEncoding extends CompressionScheme { + override def typeId = 1 + + override def encoder = new this.Encoder + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer, columnType) + } + + override def supports(columnType: ColumnType[_, _]) = columnType match { + case INT | LONG | SHORT | BYTE | STRING | BOOLEAN => true + case _ => false + } + + class Encoder extends compression.Encoder{ + private var _uncompressedSize = 0 + private var _compressedSize = 0 + + // Using `MutableRow` to store the last value to avoid boxing/unboxing cost. + private val lastValue = new GenericMutableRow(1) + private var lastRun = 0 + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = _compressedSize + + override def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) { + + val actualSize = columnType.actualSize(value) + _uncompressedSize += actualSize + + if (lastValue.isNullAt(0)) { + columnType.setField(lastValue, 0, value) + lastRun = 1 + _compressedSize += actualSize + 4 + } else { + if (columnType.getField(lastValue, 0) == value) { + lastRun += 1 + } else { + _compressedSize += actualSize + 4 + columnType.setField(lastValue, 0, value) + lastRun = 1 + } + } + } + + override def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + to.putInt(RunLengthEncoding.typeId) + + if (from.hasRemaining) { + var currentValue = columnType.extract(from) + var currentRun = 1 + + while (from.hasRemaining) { + val value = columnType.extract(from) + + if (value == currentValue) { + currentRun += 1 + } else { + // Writes current run + columnType.append(value, to) + to.putInt(currentRun) + + // Resets current run + currentValue = value + currentRun = 1 + } + } + } + + to.rewind() + to + } + } + + class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + extends compression.Decoder[T] { + + private var run = 0 + private var valueCount = 0 + private var currentValue: T#JvmType = _ + + override def next() = { + if (valueCount == run) { + currentValue = columnType.extract(buffer) + run = buffer.getInt() + valueCount = 1 + } else { + valueCount += 1 + } + + currentValue + } + + override def hasNext = buffer.hasRemaining + } +} + +private[sql] object DictionaryEncoding extends CompressionScheme { + override def typeId: Int = 2 + + // 32K unique values allowed + private val MAX_DICT_SIZE = Short.MaxValue - 1 + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder[T](buffer, columnType) + } + + override def encoder = new this.Encoder + + override def supports(columnType: ColumnType[_, _]) = columnType match { + case INT | LONG | STRING => true + case _ => false + } + + class Encoder extends compression.Encoder{ + // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary + // overflows. + private var _uncompressedSize = 0 + + // If the number of distinct elements is too large, we discard the use of dictionary encoding + // and set the overflow flag to true. + private var overflow = false + + // Total number of elements. + private var count = 0 + + // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. + private var values = new mutable.ArrayBuffer[Any](1024) + + // The dictionary that maps a value to the encoded short integer. + private val dictionary = mutable.HashMap.empty[Any, Short] + + // Size of the serialized dictionary in bytes. Initialized to 4 since we need at least an `Int` + // to store dictionary element count. + private var dictionarySize = 4 + + override def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) { + + if (!overflow) { + val actualSize = columnType.actualSize(value) + count += 1 + _uncompressedSize += actualSize + + if (!dictionary.contains(value)) { + if (dictionary.size < MAX_DICT_SIZE) { + val clone = columnType.clone(value) + values += clone + dictionarySize += actualSize + dictionary(clone) = dictionary.size.toShort + } else { + overflow = true + values.clear() + dictionary.clear() + } + } + } + } + + override def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + if (overflow) { + throw new IllegalStateException( + "Dictionary encoding should not be used because of dictionary overflow.") + } + + to.putInt(DictionaryEncoding.typeId) + .putInt(dictionary.size) + + var i = 0 + while (i < values.length) { + columnType.append(values(i).asInstanceOf[T#JvmType], to) + i += 1 + } + + while (from.hasRemaining) { + to.putShort(dictionary(columnType.extract(from))) + } + + to.rewind() + to + } + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = if (overflow) Int.MaxValue else dictionarySize + count * 2 + } + + class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + extends compression.Decoder[T] { + + private val dictionary = { + implicit val classTag = { + val mirror = runtimeMirror(getClass.getClassLoader) + ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) + } + + Array.fill(buffer.getInt()) { + columnType.extract(buffer) + } + } + + override def next() = dictionary(buffer.getShort()) + + override def hasNext = buffer.hasRemaining + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala index f853759e5a306..07f8e59b061e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala @@ -32,8 +32,8 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch lazy val cachedColumnBuffers = { val output = child.output val cached = child.execute().mapPartitions { iterator => - val columnBuilders = output.map { a => - ColumnBuilder(a.dataType.typeId, 0, a.name) + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType.typeId, 0, attribute.name) }.toArray var row: Row = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 0fa5323ba2060..8474b16070ba3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -17,22 +17,70 @@ package org.apache.spark.sql.columnar +import scala.reflect.ClassTag +import scala.util.Random + import org.scalatest.FunSuite + import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - test("Boolean") { - val stats = new BooleanColumnStats - val row = new GenericMutableRow(1) - - row(0) = false - stats.gatherStats(row, 0) - assert(stats.lowerBound === false) - assert(stats.upperBound === false) - - row(0) = true - stats.gatherStats(row, 0) - assert(stats.lowerBound === false) - assert(stats.upperBound === true) + testColumnStats[BooleanType.type, BooleanColumnStats] { + Random.nextBoolean() + } + + testColumnStats[IntegerType.type, IntColumnStats] { + Random.nextInt() + } + + testColumnStats[LongType.type, LongColumnStats] { + Random.nextLong() + } + + testColumnStats[ShortType.type, ShortColumnStats] { + (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + } + + testColumnStats[ByteType.type, ByteColumnStats] { + (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + } + + testColumnStats[DoubleType.type, DoubleColumnStats] { + Random.nextDouble() + } + + testColumnStats[FloatType.type, FloatColumnStats] { + Random.nextFloat() + } + + testColumnStats[StringType.type, StringColumnStats] { + Random.nextString(Random.nextInt(32)) + } + + def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]: ClassTag]( + mkRandomValue: => U#JvmType) { + + val columnStatsClass = implicitly[ClassTag[U]].runtimeClass + val columnStatsName = columnStatsClass.getSimpleName + + test(s"$columnStatsName: empty") { + val columnStats = columnStatsClass.newInstance().asInstanceOf[U] + assert((columnStats.lowerBound, columnStats.upperBound) === columnStats.initialBounds) + } + + test(s"$columnStatsName: non-empty") { + val columnStats = columnStatsClass.newInstance().asInstanceOf[U] + val values = Seq.fill[U#JvmType](10)(mkRandomValue) + val row = new GenericMutableRow(1) + + values.foreach { value => + row(0) = value + columnStats.gatherStats(row, 0) + } + + assert(columnStats.lowerBound === values.min(columnStats.ordering)) + assert(columnStats.upperBound === values.max(columnStats.ordering)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala index 928851a385d41..70b2e851737f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar +import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{TestData, DslQuerySuite} -class ColumnarQuerySuite extends DslQuerySuite { +class ColumnarQuerySuite extends QueryTest { import TestData._ import TestSQLContext._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 3732aaaba6687..9b62c6e57468b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) - extends BasicColumnBuilder(columnType) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) with NullableColumnBuilder object TestNullableColumnBuilder { From c298b7666c3c920b0fd30ea86a4fe047e283a086 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 1 Apr 2014 11:51:02 +0800 Subject: [PATCH 4/7] Test suites refactored --- .../spark/sql/columnar/ColumnStatsSuite.scala | 56 ++------- .../spark/sql/columnar/ColumnTypeSuite.scala | 116 ++++++------------ .../spark/sql/columnar/ColumnarTestData.scala | 70 +++++++---- .../NullableColumnAccessorSuite.scala | 12 +- .../columnar/NullableColumnBuilderSuite.scala | 11 +- 5 files changed, 108 insertions(+), 157 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 8474b16070ba3..b93f7aceda4e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -17,68 +17,32 @@ package org.apache.spark.sql.columnar -import scala.reflect.ClassTag -import scala.util.Random - import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - testColumnStats[BooleanType.type, BooleanColumnStats] { - Random.nextBoolean() - } - - testColumnStats[IntegerType.type, IntColumnStats] { - Random.nextInt() - } - - testColumnStats[LongType.type, LongColumnStats] { - Random.nextLong() - } - - testColumnStats[ShortType.type, ShortColumnStats] { - (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort - } - - testColumnStats[ByteType.type, ByteColumnStats] { - (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte - } - - testColumnStats[DoubleType.type, DoubleColumnStats] { - Random.nextDouble() - } - - testColumnStats[FloatType.type, FloatColumnStats] { - Random.nextFloat() - } - - testColumnStats[StringType.type, StringColumnStats] { - Random.nextString(Random.nextInt(32)) - } + testColumnStats(classOf[IntColumnStats], INT) - def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]: ClassTag]( - mkRandomValue: => U#JvmType) { + def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( + columnStatsClass: Class[U], + columnType: NativeColumnType[T]) { - val columnStatsClass = implicitly[ClassTag[U]].runtimeClass val columnStatsName = columnStatsClass.getSimpleName test(s"$columnStatsName: empty") { - val columnStats = columnStatsClass.newInstance().asInstanceOf[U] + val columnStats = columnStatsClass.newInstance() assert((columnStats.lowerBound, columnStats.upperBound) === columnStats.initialBounds) } test(s"$columnStatsName: non-empty") { - val columnStats = columnStatsClass.newInstance().asInstanceOf[U] - val values = Seq.fill[U#JvmType](10)(mkRandomValue) - val row = new GenericMutableRow(1) + import ColumnarTestData._ - values.foreach { value => - row(0) = value - columnStats.gatherStats(row, 0) - } + val columnStats = columnStatsClass.newInstance() + val rows = Seq.fill(10)(makeRandomRow(columnType)) + rows.foreach(columnStats.gatherStats(_, 0)) + val values = rows.map(_.head.asInstanceOf[T#JvmType]) assert(columnStats.lowerBound === values.min(columnStats.ordering)) assert(columnStats.upperBound === values.max(columnStats.ordering)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 2d431affbcfcc..2cc4d393d0b14 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -19,14 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import scala.util.Random - import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.columnar.ColumnarTestData._ import org.apache.spark.sql.execution.SparkSqlSerializer class ColumnTypeSuite extends FunSuite { + val DEFAULT_BUFFER_SIZE = 512 + val columnTypes = Seq(INT, SHORT, LONG, BYTE, DOUBLE, FLOAT, STRING, BINARY, GENERIC) test("defaultSize") { @@ -55,10 +56,8 @@ class ColumnTypeSuite extends FunSuite { } } - testNumericColumnType[BooleanType.type, Boolean]( + testNativeColumnStats[BooleanType.type]( BOOLEAN, - Array.fill(4)(Random.nextBoolean()), - ByteBuffer.allocate(32), (buffer: ByteBuffer, v: Boolean) => { buffer.put((if (v) 1 else 0).toByte) }, @@ -66,105 +65,60 @@ class ColumnTypeSuite extends FunSuite { buffer.get() == 1 }) - testNumericColumnType[IntegerType.type, Int]( + testNativeColumnStats[IntegerType.type]( INT, - Array.fill(4)(Random.nextInt()), - ByteBuffer.allocate(32), (_: ByteBuffer).putInt(_), (_: ByteBuffer).getInt) - testNumericColumnType[ShortType.type, Short]( + testNativeColumnStats[ShortType.type]( SHORT, - Array.fill(4)(Random.nextInt(Short.MaxValue).asInstanceOf[Short]), - ByteBuffer.allocate(32), (_: ByteBuffer).putShort(_), (_: ByteBuffer).getShort) - testNumericColumnType[LongType.type, Long]( + testNativeColumnStats[LongType.type]( LONG, - Array.fill(4)(Random.nextLong()), - ByteBuffer.allocate(64), (_: ByteBuffer).putLong(_), (_: ByteBuffer).getLong) - testNumericColumnType[ByteType.type, Byte]( + testNativeColumnStats[ByteType.type]( BYTE, - Array.fill(4)(Random.nextInt(Byte.MaxValue).asInstanceOf[Byte]), - ByteBuffer.allocate(64), (_: ByteBuffer).put(_), (_: ByteBuffer).get) - testNumericColumnType[DoubleType.type, Double]( + testNativeColumnStats[DoubleType.type]( DOUBLE, - Array.fill(4)(Random.nextDouble()), - ByteBuffer.allocate(64), (_: ByteBuffer).putDouble(_), (_: ByteBuffer).getDouble) - testNumericColumnType[FloatType.type, Float]( + testNativeColumnStats[FloatType.type]( FLOAT, - Array.fill(4)(Random.nextFloat()), - ByteBuffer.allocate(64), (_: ByteBuffer).putFloat(_), (_: ByteBuffer).getFloat) - test("STRING") { - val buffer = ByteBuffer.allocate(128) - val seq = Array("hello", "world", "spark", "sql") - - seq.map(_.getBytes).foreach { bytes: Array[Byte] => - buffer.putInt(bytes.length).put(bytes) - } - - buffer.rewind() - seq.foreach { s => - assert(s === STRING.extract(buffer)) - } - - buffer.rewind() - seq.foreach(STRING.append(_, buffer)) - - buffer.rewind() - seq.foreach { s => - val length = buffer.getInt - assert(length === s.getBytes.length) - + testNativeColumnStats[StringType.type]( + STRING, + (buffer: ByteBuffer, string: String) => { + val bytes = string.getBytes() + buffer.putInt(bytes.length).put(string.getBytes) + }, + (buffer: ByteBuffer) => { + val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) - assert(s === new String(bytes)) - } - } - - test("BINARY") { - val buffer = ByteBuffer.allocate(128) - val seq = Array.fill(4) { - val bytes = new Array[Byte](4) - Random.nextBytes(bytes) - bytes - } + new String(bytes) + }) - seq.foreach { bytes => + testColumnStats[BinaryType.type, Array[Byte]]( + BINARY, + (buffer: ByteBuffer, bytes: Array[Byte]) => { buffer.putInt(bytes.length).put(bytes) - } - - buffer.rewind() - seq.foreach { b => - assert(b === BINARY.extract(buffer)) - } - - buffer.rewind() - seq.foreach(BINARY.append(_, buffer)) - - buffer.rewind() - seq.foreach { b => - val length = buffer.getInt - assert(length === b.length) - + }, + (buffer: ByteBuffer) => { + val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) - assert(b === bytes) - } - } + bytes + }) test("GENERIC") { val buffer = ByteBuffer.allocate(512) @@ -188,14 +142,22 @@ class ColumnTypeSuite extends FunSuite { assert(obj === SparkSqlSerializer.deserialize(GENERIC.extract(buffer))) } - def testNumericColumnType[T <: DataType, JvmType]( + def testNativeColumnStats[T <: NativeType]( + columnType: NativeColumnType[T], + putter: (ByteBuffer, T#JvmType) => Unit, + getter: (ByteBuffer) => T#JvmType) { + + testColumnStats[T, T#JvmType](columnType, putter, getter) + } + + def testColumnStats[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], - seq: Seq[JvmType], - buffer: ByteBuffer, putter: (ByteBuffer, JvmType) => Unit, getter: (ByteBuffer) => JvmType) { + val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE) val columnTypeName = columnType.getClass.getSimpleName.stripSuffix("$") + val seq = (0 until 4).map(_ => makeRandomValue(columnType)) test(s"$columnTypeName.extract") { buffer.rewind() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala index ddcdede8d1a4a..6f019b03a071f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala @@ -19,37 +19,57 @@ package org.apache.spark.sql.columnar import scala.util.Random +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.DataType -// TODO Enrich test data object ColumnarTestData { - object GenericMutableRow { - def apply(values: Any*) = { - val row = new GenericMutableRow(values.length) - row.indices.foreach { i => - row(i) = values(i) - } - row + def makeNullRow(length: Int) = { + val row = new GenericMutableRow(length) + (0 until length).foreach(row.setNullAt) + row + } + + def makeRandomValue[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]): JvmType = { + def randomBytes(length: Int) = { + val bytes = new Array[Byte](length) + Random.nextBytes(bytes) + bytes } + + (columnType match { + case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + case INT => Random.nextInt() + case LONG => Random.nextLong() + case FLOAT => Random.nextFloat() + case DOUBLE => Random.nextDouble() + case STRING => Random.nextString(Random.nextInt(32)) + case BOOLEAN => Random.nextBoolean() + case BINARY => randomBytes(Random.nextInt(32)) + case _ => + // Using an random one-element map instead of an arbitrary object + Map(Random.nextInt() -> Random.nextString(Random.nextInt(32))) + }).asInstanceOf[JvmType] } - def randomBytes(length: Int) = { - val bytes = new Array[Byte](length) - Random.nextBytes(bytes) - bytes + def makeRandomValues( + head: ColumnType[_ <: DataType, _], + tail: ColumnType[_ <: DataType, _]*): Seq[Any] = makeRandomValues(Seq(head) ++ tail) + + def makeRandomValues(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Seq[Any] = { + columnTypes.map(makeRandomValue(_)) } - val nonNullRandomRow = GenericMutableRow( - Random.nextInt(), - Random.nextLong(), - Random.nextFloat(), - Random.nextDouble(), - Random.nextBoolean(), - Random.nextInt(Byte.MaxValue).asInstanceOf[Byte], - Random.nextInt(Short.MaxValue).asInstanceOf[Short], - Random.nextString(Random.nextInt(64)), - randomBytes(Random.nextInt(64)), - Map(Random.nextInt() -> Random.nextString(4))) - - val nullRow = GenericMutableRow(Seq.fill(10)(null): _*) + def makeRandomRow( + head: ColumnType[_ <: DataType, _], + tail: ColumnType[_ <: DataType, _]*): Row = makeRandomRow(Seq(head) ++ tail) + + def makeRandomRow(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Row = { + val row = new GenericMutableRow(columnTypes.length) + makeRandomValues(columnTypes).zipWithIndex.foreach { case (value, index) => + row(index) = value + } + row + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 61f7338913791..8558baef06b3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -47,19 +47,21 @@ class NullableColumnAccessorSuite extends FunSuite { def testNullableColumnAccessor[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + val nullRow = makeNullRow(1) - test(s"$typeName accessor: empty column") { + test(s"Nullable $typeName accessor: empty column") { val builder = TestNullableColumnBuilder(columnType) val accessor = TestNullableColumnAccessor(builder.build(), columnType) assert(!accessor.hasNext) } - test(s"$typeName accessor: access null values") { + test(s"Nullable $typeName accessor: access null values") { val builder = TestNullableColumnBuilder(columnType) + val randomRow = makeRandomRow(columnType) (0 until 4).foreach { _ => - builder.appendFrom(nonNullRandomRow, columnType.typeId) - builder.appendFrom(nullRow, columnType.typeId) + builder.appendFrom(randomRow, 0) + builder.appendFrom(nullRow, 0) } val accessor = TestNullableColumnAccessor(builder.build(), columnType) @@ -67,7 +69,7 @@ class NullableColumnAccessorSuite extends FunSuite { (0 until 4).foreach { _ => accessor.extractTo(row, 0) - assert(row(0) === nonNullRandomRow(columnType.typeId)) + assert(row(0) === randomRow(0)) accessor.extractTo(row, 0) assert(row(0) === null) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 9b62c6e57468b..3c6c9a69b4d90 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -57,9 +57,10 @@ class NullableColumnBuilderSuite extends FunSuite { test(s"$typeName column builder: buffer size auto growth") { val columnBuilder = TestNullableColumnBuilder(columnType) + val randomRow = makeRandomRow(columnType) (0 until 4) foreach { _ => - columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) + columnBuilder.appendFrom(randomRow, 0) } val buffer = columnBuilder.build() @@ -72,10 +73,12 @@ class NullableColumnBuilderSuite extends FunSuite { test(s"$typeName column builder: null values") { val columnBuilder = TestNullableColumnBuilder(columnType) + val randomRow = makeRandomRow(columnType) + val nullRow = makeNullRow(1) (0 until 4) foreach { _ => - columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) - columnBuilder.appendFrom(nullRow, columnType.typeId) + columnBuilder.appendFrom(randomRow, 0) + columnBuilder.appendFrom(nullRow, 0) } val buffer = columnBuilder.build() @@ -94,7 +97,7 @@ class NullableColumnBuilderSuite extends FunSuite { } else { columnType.extract(buffer) } - assert(actual === nonNullRandomRow(columnType.typeId)) + assert(actual === randomRow.head) } assert(!buffer.hasRemaining) From 50344531c5a6846c87d13093d23607da0a0803b7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 1 Apr 2014 21:16:57 +0800 Subject: [PATCH 5/7] Bug fix, more tests, and more refactoring * Added test suites for RunLengthEncoding and DictionaryEncoding * Completed ColumnStatsSuite * Bug fix: RunLengthEncoding didn't encode the last run * Refactored some test related code for clarity --- .../CompressibleColumnBuilder.scala | 6 +- .../compression/compressionSchemes.scala | 6 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 11 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 2 +- ...TestData.scala => ColumnarTestUtils.scala} | 31 ++++- .../NullableColumnAccessorSuite.scala | 2 +- .../columnar/NullableColumnBuilderSuite.scala | 2 +- .../compression/DictionaryEncodingSuite.scala | 113 +++++++++++++++ .../compression/RunLengthEncodingSuite.scala | 131 ++++++++++++++++++ .../TestCompressibleColumnBuilder.scala | 43 ++++++ 10 files changed, 335 insertions(+), 12 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/columnar/{ColumnarTestData.scala => ColumnarTestUtils.scala} (75%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index bb15ca191069d..3ac4b358ddf83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -41,15 +41,15 @@ import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} * }}} */ private[sql] trait CompressibleColumnBuilder[T <: NativeType] - extends ColumnBuilder with WithCompressionSchemes with Logging { + extends ColumnBuilder with Logging { - this: NativeColumnBuilder[T] => + this: NativeColumnBuilder[T] with WithCompressionSchemes => import CompressionScheme._ val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder) - private def isWorthCompressing(encoder: Encoder) = { + protected def isWorthCompressing(encoder: Encoder) = { encoder.compressionRatio < 0.8 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index f77e15f1036e6..053e1b6fe7e9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -129,7 +129,7 @@ private[sql] object RunLengthEncoding extends CompressionScheme { currentRun += 1 } else { // Writes current run - columnType.append(value, to) + columnType.append(currentValue, to) to.putInt(currentRun) // Resets current run @@ -137,6 +137,10 @@ private[sql] object RunLengthEncoding extends CompressionScheme { currentRun = 1 } } + + // Writes the last run + columnType.append(currentValue, to) + to.putInt(currentRun) } to.rewind() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index b93f7aceda4e5..3e9974daea986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -22,7 +22,14 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - testColumnStats(classOf[IntColumnStats], INT) + testColumnStats(classOf[BooleanColumnStats], BOOLEAN) + testColumnStats(classOf[ByteColumnStats], BYTE) + testColumnStats(classOf[ShortColumnStats], SHORT) + testColumnStats(classOf[IntColumnStats], INT) + testColumnStats(classOf[LongColumnStats], LONG) + testColumnStats(classOf[FloatColumnStats], FLOAT) + testColumnStats(classOf[DoubleColumnStats], DOUBLE) + testColumnStats(classOf[StringColumnStats], STRING) def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( columnStatsClass: Class[U], @@ -36,7 +43,7 @@ class ColumnStatsSuite extends FunSuite { } test(s"$columnStatsName: non-empty") { - import ColumnarTestData._ + import ColumnarTestUtils._ val columnStats = columnStatsClass.newInstance() val rows = Seq.fill(10)(makeRandomRow(columnType)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 2cc4d393d0b14..1a98ec270c03b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.columnar.ColumnarTestData._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer class ColumnTypeSuite extends FunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala similarity index 75% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala rename to sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 6f019b03a071f..04bdc43d95328 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.columnar +import scala.collection.immutable.HashSet import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.types.{DataType, NativeType} -object ColumnarTestData { +object ColumnarTestUtils { def makeNullRow(length: Int) = { val row = new GenericMutableRow(length) (0 until length).foreach(row.setNullAt) @@ -48,7 +49,7 @@ object ColumnarTestData { case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) case _ => - // Using an random one-element map instead of an arbitrary object + // Using a random one-element map instead of an arbitrary object Map(Random.nextInt() -> Random.nextString(Random.nextInt(32))) }).asInstanceOf[JvmType] } @@ -61,6 +62,15 @@ object ColumnarTestData { columnTypes.map(makeRandomValue(_)) } + def makeUniqueRandomValues[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType], + count: Int): Seq[JvmType] = { + + Iterator.iterate(HashSet.empty[JvmType]) { set => + set + Iterator.continually(makeRandomValue(columnType)).filterNot(set.contains).next() + }.drop(count).next().toSeq + } + def makeRandomRow( head: ColumnType[_ <: DataType, _], tail: ColumnType[_ <: DataType, _]*): Row = makeRandomRow(Seq(head) ++ tail) @@ -72,4 +82,19 @@ object ColumnarTestData { } row } + + def makeUniqueValuesAndSingleValueRows[T <: NativeType]( + columnType: NativeColumnType[T], + count: Int) = { + + val values = makeUniqueRandomValues(columnType, count) + val rows = values.map { value => + val row = new GenericMutableRow(1) + row(0) = value + row + } + + (values, rows) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 8558baef06b3a..9b12441b99566 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -39,7 +39,7 @@ object TestNullableColumnAccessor { } class NullableColumnAccessorSuite extends FunSuite { - import ColumnarTestData._ + import ColumnarTestUtils._ Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { testNullableColumnAccessor(_) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 3c6c9a69b4d90..aaeeb1272b03f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -35,7 +35,7 @@ object TestNullableColumnBuilder { } class NullableColumnBuilderSuite extends FunSuite { - import ColumnarTestData._ + import ColumnarTestUtils._ Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { testNullableColumnBuilder(_) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala new file mode 100644 index 0000000000000..3a6cc2f2ba56e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import java.nio.ByteBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +class DictionaryEncodingSuite extends FunSuite { + testDictionaryEncoding(new IntColumnStats, INT) + testDictionaryEncoding(new LongColumnStats, LONG) + testDictionaryEncoding(new StringColumnStats, STRING) + + val schemeName = DictionaryEncoding.getClass.getSimpleName.stripSuffix("$") + + def testDictionaryEncoding[T <: NativeType]( + columnStats: NativeColumnStats[T], + columnType: NativeColumnType[T]) { + + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + def buildDictionary(buffer: ByteBuffer) = { + (0 until buffer.getInt()).map(columnType.extract(buffer) -> _.toShort).toMap + } + + test(s"$schemeName with $typeName: simple case") { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) + + builder.initialize(0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 bytes for dictionary size + val dictionarySize = 4 + values.map(columnType.actualSize).sum + val compressedSize = dictionarySize + 2 * 4 + // 4 bytes for compression scheme type ID + assert(buffer.capacity === headerSize + 4 + compressedSize) + + // Skips column header + buffer.position(headerSize) + // Checks compression scheme ID + assert(buffer.getInt() === DictionaryEncoding.typeId) + + val dictionary = buildDictionary(buffer) + assert(dictionary(values(0)) === (0: Short)) + assert(dictionary(values(1)) === (1: Short)) + + assert(buffer.getShort() === (0: Short)) + assert(buffer.getShort() === (1: Short)) + assert(buffer.getShort() === (0: Short)) + assert(buffer.getShort() === (1: Short)) + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = new DictionaryEncoding.Decoder[T](buffer, columnType) + + assert(decoder.next() === values(0)) + assert(decoder.next() === values(1)) + assert(decoder.next() === values(0)) + assert(decoder.next() === values(1)) + assert(!decoder.hasNext) + } + } + + test(s"$schemeName: overflow") { + val builder = TestCompressibleColumnBuilder(new IntColumnStats, INT, DictionaryEncoding) + builder.initialize(0) + + (0 to Short.MaxValue).foreach { n => + val row = new GenericMutableRow(1) + row.setInt(0, n) + builder.appendFrom(row, 0) + } + + intercept[Throwable] { + builder.build() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala new file mode 100644 index 0000000000000..ac9bc222aad14 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ + +class RunLengthEncodingSuite extends FunSuite { + testRunLengthEncoding(new BooleanColumnStats, BOOLEAN) + testRunLengthEncoding(new ByteColumnStats, BYTE) + testRunLengthEncoding(new ShortColumnStats, SHORT) + testRunLengthEncoding(new IntColumnStats, INT) + testRunLengthEncoding(new LongColumnStats, LONG) + testRunLengthEncoding(new StringColumnStats, STRING) + + def testRunLengthEncoding[T <: NativeType]( + columnStats: NativeColumnStats[T], + columnType: NativeColumnType[T]) { + + val schemeName = RunLengthEncoding.getClass.getSimpleName.stripSuffix("$") + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + test(s"$schemeName with $typeName: simple case") { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) + + builder.initialize(0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + builder.appendFrom(rows(1), 0) + + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 bytes each run for run length + val compressedSize = values.map(columnType.actualSize(_) + 4).sum + // 4 bytes for compression scheme type ID + assert(buffer.capacity === headerSize + 4 + compressedSize) + + // Skips column header + buffer.position(headerSize) + // Checks compression scheme ID + assert(buffer.getInt() === RunLengthEncoding.typeId) + + assert(columnType.extract(buffer) === values(0)) + assert(buffer.getInt() === 2) + assert(columnType.extract(buffer) === values(1)) + assert(buffer.getInt() === 2) + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + + assert(decoder.next() === values(0)) + assert(decoder.next() === values(0)) + assert(decoder.next() === values(1)) + assert(decoder.next() === values(1)) + assert(!decoder.hasNext) + } + + test(s"$schemeName with $typeName: run length == 1") { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) + + builder.initialize(0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 bytes each run for run length + val compressedSize = values.map(columnType.actualSize(_) + 4).sum + // 4 bytes for compression scheme type ID + assert(buffer.capacity === headerSize + 4 + compressedSize) + + // Skips column header + buffer.position(headerSize) + // Checks compression scheme ID + assert(buffer.getInt() === RunLengthEncoding.typeId) + + assert(columnType.extract(buffer) === values(0)) + assert(buffer.getInt() === 1) + assert(columnType.extract(buffer) === values(1)) + assert(buffer.getInt() === 1) + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + + assert(decoder.next() === values(0)) + assert(decoder.next() === values(1)) + assert(!decoder.hasNext) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala new file mode 100644 index 0000000000000..e0ec812863dcf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ + +class TestCompressibleColumnBuilder[T <: NativeType]( + override val columnStats: NativeColumnStats[T], + override val columnType: NativeColumnType[T], + override val schemes: Seq[CompressionScheme]) + extends NativeColumnBuilder(columnStats, columnType) + with NullableColumnBuilder + with CompressibleColumnBuilder[T] { + + override protected def isWorthCompressing(encoder: Encoder) = true +} + +object TestCompressibleColumnBuilder { + def apply[T <: NativeType]( + columnStats: NativeColumnStats[T], + columnType: NativeColumnType[T], + scheme: CompressionScheme) = { + + new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) + } +} + From d3a4fa9d0b1b0a71eab6ec073ded8dc12e791031 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 1 Apr 2014 23:14:46 +0800 Subject: [PATCH 6/7] Removed Ordering[T] in ColumnStats for better performance --- .../spark/sql/columnar/ColumnStats.scala | 185 ++++++++++++++---- .../compression/compressionSchemes.scala | 2 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 6 +- 3 files changed, 153 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 43096ee0da945..c7344dd9e1cd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -69,36 +69,13 @@ private[sql] sealed abstract class NativeColumnStats[T <: NativeType] protected var (_lower, _upper) = initialBounds - val ordering: Ordering[JvmType] - def initialBounds: (JvmType, JvmType) protected def columnType: NativeColumnType[T] - override def lowerBound = _lower - - override def upperBound = _upper + override def lowerBound: T#JvmType = _lower - override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (upperBound == null || ordering.gt(field, upperBound)) _upper = field - if (lowerBound == null || ordering.lt(field, lowerBound)) _lower = field - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - ordering.lteq(lowerBound, field) && ordering.lteq(field, upperBound) - } - - override def isAbove(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - ordering.lt(field, upperBound) - } - - override def isBelow(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - ordering.lt(lowerBound, field) - } + override def upperBound: T#JvmType = _upper override def isAtOrAbove(row: Row, ordinal: Int) = { contains(row, ordinal) || isAbove(row, ordinal) @@ -132,33 +109,141 @@ private[sql] abstract class BasicColumnStats[T <: NativeType]( extends NativeColumnStats[T] private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (true, false) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } } private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (Byte.MaxValue, Byte.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } } private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (Short.MaxValue, Short.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } } private[sql] class LongColumnStats extends BasicColumnStats(LONG) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (Long.MaxValue, Long.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } } private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (Double.MaxValue, Double.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } } private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (Float.MaxValue, Float.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } } object IntColumnStats { @@ -181,10 +266,22 @@ private[sql] class IntColumnStats extends BasicColumnStats(INT) { def isOrdered = isAscending || isDescending def maxDelta = _maxDelta - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (Int.MaxValue, Int.MinValue) - override def gatherStats(row: Row, ordinal: Int) = { + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { val field = columnType.getField(row, ordinal) if (field > upperBound) _upper = field @@ -221,18 +318,32 @@ private[sql] class IntColumnStats extends BasicColumnStats(INT) { } private[sql] class StringColumnStats extends BasicColumnStats(STRING) { - override val ordering = implicitly[Ordering[JvmType]] override def initialBounds = (null, null) + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field + if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field + } + override def contains(row: Row, ordinal: Int) = { - !(upperBound eq null) && super.contains(row, ordinal) + !(upperBound eq null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 + } } override def isAbove(row: Row, ordinal: Int) = { - !(upperBound eq null) && super.isAbove(row, ordinal) + !(upperBound eq null) && { + val field = columnType.getField(row, ordinal) + field.compareTo(upperBound) < 0 + } } override def isBelow(row: Row, ordinal: Int) = { - !(lowerBound eq null) && super.isBelow(row, ordinal) + !(lowerBound eq null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) < 0 + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 053e1b6fe7e9e..50ebe6c907f32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -77,7 +77,7 @@ private[sql] object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder extends compression.Encoder{ + class Encoder extends compression.Encoder { private var _uncompressedSize = 0 private var _compressedSize = 0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 3e9974daea986..f830bb974627b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -50,8 +50,10 @@ class ColumnStatsSuite extends FunSuite { rows.foreach(columnStats.gatherStats(_, 0)) val values = rows.map(_.head.asInstanceOf[T#JvmType]) - assert(columnStats.lowerBound === values.min(columnStats.ordering)) - assert(columnStats.upperBound === values.max(columnStats.ordering)) + val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] + + assert(columnStats.lowerBound === values.min(ordering)) + assert(columnStats.upperBound === values.max(ordering)) } } } From ed71bbd38f389dcdc11949513a02fe11ff7bdb6a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 2 Apr 2014 20:54:24 +0800 Subject: [PATCH 7/7] Addressed all PR comments by @marmbrus https://github.com/apache/spark/pull/285 --- .../spark/sql/columnar/ColumnStats.scala | 15 +- .../spark/sql/columnar/ColumnType.scala | 4 +- ....scala => InMemoryColumnarTableScan.scala} | 5 +- .../compression/CompressionScheme.scala | 4 +- .../compression/compressionSchemes.scala | 7 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 8 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 134 +++++++++--------- .../NullableColumnAccessorSuite.scala | 6 +- .../columnar/NullableColumnBuilderSuite.scala | 28 ++-- .../compression/DictionaryEncodingSuite.scala | 42 +++--- .../compression/RunLengthEncodingSuite.scala | 49 ++++--- 11 files changed, 156 insertions(+), 146 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/columnar/{inMemoryColumnarOperators.scala => InMemoryColumnarTableScan.scala} (94%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index c7344dd9e1cd3..30c6bdc7912fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ -private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable{ +private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable { /** * Closed lower bound of this column. */ @@ -246,7 +246,7 @@ private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { } } -object IntColumnStats { +private[sql] object IntColumnStats { val UNINITIALIZED = 0 val INITIALIZED = 1 val ASCENDING = 2 @@ -254,6 +254,17 @@ object IntColumnStats { val UNORDERED = 4 } +/** + * Statistical information for `Int` columns. More information is collected since `Int` is + * frequently used. Extra information include: + * + * - Ordering state (ascending/descending/unordered), may be used to decide whether binary search + * is applicable when searching elements. + * - Maximum delta between adjacent elements, may be used to guide the `IntDelta` compression + * scheme. + * + * (This two kinds of information are not used anywhere yet and might be removed later.) + */ private[sql] class IntColumnStats extends BasicColumnStats(INT) { import IntColumnStats._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index a761c42b2aba2..5be76890afe31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -71,6 +71,8 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( * Creates a duplicated copy of the value. */ def clone(v: JvmType): JvmType = v + + override def toString = getClass.getSimpleName.stripSuffix("$") } private[sql] abstract class NativeColumnType[T <: NativeType]( @@ -258,7 +260,7 @@ private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { } private[sql] object ColumnType { - implicit def dataTypeToColumnType(dataType: DataType): ColumnType[_, _] = { + def apply(dataType: DataType): ColumnType[_, _] = { dataType match { case IntegerType => INT case LongType => LONG diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala rename to sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 07f8e59b061e2..8a24733047423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -21,9 +21,6 @@ import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.execution.{SparkPlan, LeafNode} import org.apache.spark.sql.Row -/* Implicit conversions */ -import org.apache.spark.sql.columnar.ColumnType._ - private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) extends LeafNode { @@ -33,7 +30,7 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch val output = child.output val cached = child.execute().mapPartitions { iterator => val columnBuilders = output.map { attribute => - ColumnBuilder(attribute.dataType.typeId, 0, attribute.name) + ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name) }.toArray var row: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index be2f359553020..d3a4ac8df926b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -58,7 +58,9 @@ private[sql] trait WithCompressionSchemes { } private[sql] trait AllCompressionSchemes extends WithCompressionSchemes { - override val schemes = Seq(PassThrough, RunLengthEncoding, DictionaryEncoding) + override val schemes: Seq[CompressionScheme] = { + Seq(PassThrough, RunLengthEncoding, DictionaryEncoding) + } } private[sql] object CompressionScheme { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 50ebe6c907f32..dc2c153faf8ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ -private[sql] object PassThrough extends CompressionScheme { +private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 override def supports(columnType: ColumnType[_, _]) = true @@ -63,7 +63,7 @@ private[sql] object PassThrough extends CompressionScheme { } } -private[sql] object RunLengthEncoding extends CompressionScheme { +private[sql] case object RunLengthEncoding extends CompressionScheme { override def typeId = 1 override def encoder = new this.Encoder @@ -171,7 +171,7 @@ private[sql] object RunLengthEncoding extends CompressionScheme { } } -private[sql] object DictionaryEncoding extends CompressionScheme { +private[sql] case object DictionaryEncoding extends CompressionScheme { override def typeId: Int = 2 // 32K unique values allowed @@ -270,6 +270,7 @@ private[sql] object DictionaryEncoding extends CompressionScheme { extends compression.Decoder[T] { private val dictionary = { + // TODO Can we clean up this mess? Maybe move this to `DataType`? implicit val classTag = { val mirror = runtimeMirror(getClass.getClassLoader) ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index f830bb974627b..78640b876d4aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -39,7 +39,9 @@ class ColumnStatsSuite extends FunSuite { test(s"$columnStatsName: empty") { val columnStats = columnStatsClass.newInstance() - assert((columnStats.lowerBound, columnStats.upperBound) === columnStats.initialBounds) + expectResult(columnStats.initialBounds, "Wrong initial bounds") { + (columnStats.lowerBound, columnStats.upperBound) + } } test(s"$columnStatsName: non-empty") { @@ -52,8 +54,8 @@ class ColumnStatsSuite extends FunSuite { val values = rows.map(_.head.asInstanceOf[T#JvmType]) val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] - assert(columnStats.lowerBound === values.min(ordering)) - assert(columnStats.upperBound === values.max(ordering)) + expectResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) + expectResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 1a98ec270c03b..1d3608ed2d9ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -28,35 +28,46 @@ import org.apache.spark.sql.execution.SparkSqlSerializer class ColumnTypeSuite extends FunSuite { val DEFAULT_BUFFER_SIZE = 512 - val columnTypes = Seq(INT, SHORT, LONG, BYTE, DOUBLE, FLOAT, STRING, BINARY, GENERIC) - test("defaultSize") { - val defaultSize = Seq(4, 2, 8, 1, 8, 4, 8, 16, 16) + val checks = Map( + INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, + BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16) - columnTypes.zip(defaultSize).foreach { case (columnType, size) => - assert(columnType.defaultSize === size) + checks.foreach { case (columnType, expectedSize) => + expectResult(expectedSize, s"Wrong defaultSize for $columnType") { + columnType.defaultSize + } } } test("actualSize") { - val expectedSizes = Seq(4, 2, 8, 1, 8, 4, 4 + 5, 4 + 4, 4 + 11) - val actualSizes = Seq( - INT.actualSize(Int.MaxValue), - SHORT.actualSize(Short.MaxValue), - LONG.actualSize(Long.MaxValue), - BYTE.actualSize(Byte.MaxValue), - DOUBLE.actualSize(Double.MaxValue), - FLOAT.actualSize(Float.MaxValue), - STRING.actualSize("hello"), - BINARY.actualSize(new Array[Byte](4)), - GENERIC.actualSize(SparkSqlSerializer.serialize(Map(1 -> "a")))) - - expectedSizes.zip(actualSizes).foreach { case (expected, actual) => - assert(expected === actual) + def checkActualSize[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType], + value: JvmType, + expected: Int) { + + expectResult(expected, s"Wrong actualSize for $columnType") { + columnType.actualSize(value) + } } + + checkActualSize(INT, Int.MaxValue, 4) + checkActualSize(SHORT, Short.MaxValue, 2) + checkActualSize(LONG, Long.MaxValue, 8) + checkActualSize(BYTE, Byte.MaxValue, 1) + checkActualSize(DOUBLE, Double.MaxValue, 8) + checkActualSize(FLOAT, Float.MaxValue, 4) + checkActualSize(BOOLEAN, true, 1) + checkActualSize(STRING, "hello", 4 + 5) + + val binary = Array.fill[Byte](4)(0: Byte) + checkActualSize(BINARY, binary, 4 + 4) + + val generic = Map(1 -> "a") + checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11) } - testNativeColumnStats[BooleanType.type]( + testNativeColumnType[BooleanType.type]( BOOLEAN, (buffer: ByteBuffer, v: Boolean) => { buffer.put((if (v) 1 else 0).toByte) @@ -65,37 +76,19 @@ class ColumnTypeSuite extends FunSuite { buffer.get() == 1 }) - testNativeColumnStats[IntegerType.type]( - INT, - (_: ByteBuffer).putInt(_), - (_: ByteBuffer).getInt) - - testNativeColumnStats[ShortType.type]( - SHORT, - (_: ByteBuffer).putShort(_), - (_: ByteBuffer).getShort) - - testNativeColumnStats[LongType.type]( - LONG, - (_: ByteBuffer).putLong(_), - (_: ByteBuffer).getLong) - - testNativeColumnStats[ByteType.type]( - BYTE, - (_: ByteBuffer).put(_), - (_: ByteBuffer).get) - - testNativeColumnStats[DoubleType.type]( - DOUBLE, - (_: ByteBuffer).putDouble(_), - (_: ByteBuffer).getDouble) - - testNativeColumnStats[FloatType.type]( - FLOAT, - (_: ByteBuffer).putFloat(_), - (_: ByteBuffer).getFloat) - - testNativeColumnStats[StringType.type]( + testNativeColumnType[IntegerType.type](INT, _.putInt(_), _.getInt) + + testNativeColumnType[ShortType.type](SHORT, _.putShort(_), _.getShort) + + testNativeColumnType[LongType.type](LONG, _.putLong(_), _.getLong) + + testNativeColumnType[ByteType.type](BYTE, _.put(_), _.get) + + testNativeColumnType[DoubleType.type](DOUBLE, _.putDouble(_), _.getDouble) + + testNativeColumnType[FloatType.type](FLOAT, _.putFloat(_), _.getFloat) + + testNativeColumnType[StringType.type]( STRING, (buffer: ByteBuffer, string: String) => { val bytes = string.getBytes() @@ -108,7 +101,7 @@ class ColumnTypeSuite extends FunSuite { new String(bytes) }) - testColumnStats[BinaryType.type, Array[Byte]]( + testColumnType[BinaryType.type, Array[Byte]]( BINARY, (buffer: ByteBuffer, bytes: Array[Byte]) => { buffer.putInt(bytes.length).put(bytes) @@ -131,51 +124,58 @@ class ColumnTypeSuite extends FunSuite { val length = buffer.getInt() assert(length === serializedObj.length) - val bytes = new Array[Byte](length) - buffer.get(bytes, 0, length) - assert(obj === SparkSqlSerializer.deserialize(bytes)) + expectResult(obj, "Deserialized object didn't equal to the original object") { + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + SparkSqlSerializer.deserialize(bytes) + } buffer.rewind() buffer.putInt(serializedObj.length).put(serializedObj) - buffer.rewind() - assert(obj === SparkSqlSerializer.deserialize(GENERIC.extract(buffer))) + expectResult(obj, "Deserialized object didn't equal to the original object") { + buffer.rewind() + SparkSqlSerializer.deserialize(GENERIC.extract(buffer)) + } } - def testNativeColumnStats[T <: NativeType]( + def testNativeColumnType[T <: NativeType]( columnType: NativeColumnType[T], putter: (ByteBuffer, T#JvmType) => Unit, getter: (ByteBuffer) => T#JvmType) { - testColumnStats[T, T#JvmType](columnType, putter, getter) + testColumnType[T, T#JvmType](columnType, putter, getter) } - def testColumnStats[T <: DataType, JvmType]( + def testColumnType[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], putter: (ByteBuffer, JvmType) => Unit, getter: (ByteBuffer) => JvmType) { val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE) - val columnTypeName = columnType.getClass.getSimpleName.stripSuffix("$") val seq = (0 until 4).map(_ => makeRandomValue(columnType)) - test(s"$columnTypeName.extract") { + test(s"$columnType.extract") { buffer.rewind() seq.foreach(putter(buffer, _)) buffer.rewind() - seq.foreach { i => - assert(i === columnType.extract(buffer)) + seq.foreach { expected => + assert( + expected === columnType.extract(buffer), + "Extracted value didn't equal to the original one") } } - test(s"$columnTypeName.append") { + test(s"$columnType.append") { buffer.rewind() seq.foreach(columnType.append(_, buffer)) buffer.rewind() - seq.foreach { i => - assert(i === getter(buffer)) + seq.foreach { expected => + assert( + expected === getter(buffer), + "Extracted value didn't equal to the original one") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 9b12441b99566..4a21eb6201a69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -49,13 +49,13 @@ class NullableColumnAccessorSuite extends FunSuite { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") val nullRow = makeNullRow(1) - test(s"Nullable $typeName accessor: empty column") { + test(s"Nullable $typeName column accessor: empty column") { val builder = TestNullableColumnBuilder(columnType) val accessor = TestNullableColumnAccessor(builder.build(), columnType) assert(!accessor.hasNext) } - test(s"Nullable $typeName accessor: access null values") { + test(s"Nullable $typeName column accessor: access null values") { val builder = TestNullableColumnBuilder(columnType) val randomRow = makeRandomRow(columnType) @@ -72,7 +72,7 @@ class NullableColumnAccessorSuite extends FunSuite { assert(row(0) === randomRow(0)) accessor.extractTo(row, 0) - assert(row(0) === null) + assert(row.isNullAt(0)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index aaeeb1272b03f..d9d1e1bfddb75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -48,10 +48,8 @@ class NullableColumnBuilderSuite extends FunSuite { val columnBuilder = TestNullableColumnBuilder(columnType) val buffer = columnBuilder.build() - // For column type ID - assert(buffer.getInt() === columnType.typeId) - // For null count - assert(buffer.getInt() === 0) + expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + expectResult(0, "Wrong null count")(buffer.getInt()) assert(!buffer.hasRemaining) } @@ -59,16 +57,14 @@ class NullableColumnBuilderSuite extends FunSuite { val columnBuilder = TestNullableColumnBuilder(columnType) val randomRow = makeRandomRow(columnType) - (0 until 4) foreach { _ => + (0 until 4).foreach { _ => columnBuilder.appendFrom(randomRow, 0) } val buffer = columnBuilder.build() - // For column type ID - assert(buffer.getInt() === columnType.typeId) - // For null count - assert(buffer.getInt() === 0) + expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + expectResult(0, "Wrong null count")(buffer.getInt()) } test(s"$typeName column builder: null values") { @@ -76,19 +72,18 @@ class NullableColumnBuilderSuite extends FunSuite { val randomRow = makeRandomRow(columnType) val nullRow = makeNullRow(1) - (0 until 4) foreach { _ => + (0 until 4).foreach { _ => columnBuilder.appendFrom(randomRow, 0) columnBuilder.appendFrom(nullRow, 0) } val buffer = columnBuilder.build() - // For column type ID - assert(buffer.getInt() === columnType.typeId) - // For null count - assert(buffer.getInt() === 4) + expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + expectResult(4, "Wrong null count")(buffer.getInt()) + // For null positions - (1 to 7 by 2).foreach(i => assert(buffer.getInt() === i)) + (1 to 7 by 2).foreach(expectResult(_, "Wrong null position")(buffer.getInt())) // For non-null values (0 until 4).foreach { _ => @@ -97,7 +92,8 @@ class NullableColumnBuilderSuite extends FunSuite { } else { columnType.extract(buffer) } - assert(actual === randomRow.head) + + assert(actual === randomRow(0), "Extracted value didn't equal to the original one") } assert(!buffer.hasRemaining) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 3a6cc2f2ba56e..184691ab5b46a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -31,8 +31,6 @@ class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new LongColumnStats, LONG) testDictionaryEncoding(new StringColumnStats, STRING) - val schemeName = DictionaryEncoding.getClass.getSimpleName.stripSuffix("$") - def testDictionaryEncoding[T <: NativeType]( columnStats: NativeColumnStats[T], columnType: NativeColumnType[T]) { @@ -43,7 +41,7 @@ class DictionaryEncodingSuite extends FunSuite { (0 until buffer.getInt()).map(columnType.extract(buffer) -> _.toShort).toMap } - test(s"$schemeName with $typeName: simple case") { + test(s"$DictionaryEncoding with $typeName: simple case") { // ------------- // Tests encoder // ------------- @@ -59,25 +57,25 @@ class DictionaryEncodingSuite extends FunSuite { val buffer = builder.build() val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 bytes for dictionary size + // 4 extra bytes for dictionary size val dictionarySize = 4 + values.map(columnType.actualSize).sum + // 4 `Short`s, 2 bytes each val compressedSize = dictionarySize + 2 * 4 - // 4 bytes for compression scheme type ID - assert(buffer.capacity === headerSize + 4 + compressedSize) + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - // Checks compression scheme ID - assert(buffer.getInt() === DictionaryEncoding.typeId) + expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val dictionary = buildDictionary(buffer) - assert(dictionary(values(0)) === (0: Short)) - assert(dictionary(values(1)) === (1: Short)) + Array[Short](0, 1).foreach { i => + expectResult(i, "Wrong dictionary entry")(dictionary(values(i))) + } - assert(buffer.getShort() === (0: Short)) - assert(buffer.getShort() === (1: Short)) - assert(buffer.getShort() === (0: Short)) - assert(buffer.getShort() === (1: Short)) + Array[Short](0, 1, 0, 1).foreach { + expectResult(_, "Wrong column element value")(buffer.getShort()) + } // ------------- // Tests decoder @@ -88,15 +86,15 @@ class DictionaryEncodingSuite extends FunSuite { val decoder = new DictionaryEncoding.Decoder[T](buffer, columnType) - assert(decoder.next() === values(0)) - assert(decoder.next() === values(1)) - assert(decoder.next() === values(0)) - assert(decoder.next() === values(1)) + Array[Short](0, 1, 0, 1).foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + assert(!decoder.hasNext) } } - test(s"$schemeName: overflow") { + test(s"$DictionaryEncoding: overflow") { val builder = TestCompressibleColumnBuilder(new IntColumnStats, INT, DictionaryEncoding) builder.initialize(0) @@ -106,8 +104,10 @@ class DictionaryEncodingSuite extends FunSuite { builder.appendFrom(row, 0) } - intercept[Throwable] { - builder.build() + withClue("Dictionary overflowed, encoding should fail") { + intercept[Throwable] { + builder.build() + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index ac9bc222aad14..2089ad120d4f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -35,10 +35,9 @@ class RunLengthEncodingSuite extends FunSuite { columnStats: NativeColumnStats[T], columnType: NativeColumnType[T]) { - val schemeName = RunLengthEncoding.getClass.getSimpleName.stripSuffix("$") val typeName = columnType.getClass.getSimpleName.stripSuffix("$") - test(s"$schemeName with $typeName: simple case") { + test(s"$RunLengthEncoding with $typeName: simple case") { // ------------- // Tests encoder // ------------- @@ -54,20 +53,19 @@ class RunLengthEncodingSuite extends FunSuite { val buffer = builder.build() val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 bytes each run for run length + // 4 extra bytes each run for run length val compressedSize = values.map(columnType.actualSize(_) + 4).sum - // 4 bytes for compression scheme type ID - assert(buffer.capacity === headerSize + 4 + compressedSize) + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - // Checks compression scheme ID - assert(buffer.getInt() === RunLengthEncoding.typeId) + expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - assert(columnType.extract(buffer) === values(0)) - assert(buffer.getInt() === 2) - assert(columnType.extract(buffer) === values(1)) - assert(buffer.getInt() === 2) + Array(0, 1).foreach { i => + expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) + expectResult(2, "Wrong run length")(buffer.getInt()) + } // ------------- // Tests decoder @@ -78,14 +76,14 @@ class RunLengthEncodingSuite extends FunSuite { val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) - assert(decoder.next() === values(0)) - assert(decoder.next() === values(0)) - assert(decoder.next() === values(1)) - assert(decoder.next() === values(1)) + Array(0, 0, 1, 1).foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + assert(!decoder.hasNext) } - test(s"$schemeName with $typeName: run length == 1") { + test(s"$RunLengthEncoding with $typeName: run length == 1") { // ------------- // Tests encoder // ------------- @@ -102,17 +100,16 @@ class RunLengthEncodingSuite extends FunSuite { // 4 bytes each run for run length val compressedSize = values.map(columnType.actualSize(_) + 4).sum // 4 bytes for compression scheme type ID - assert(buffer.capacity === headerSize + 4 + compressedSize) + expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - // Checks compression scheme ID - assert(buffer.getInt() === RunLengthEncoding.typeId) + expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - assert(columnType.extract(buffer) === values(0)) - assert(buffer.getInt() === 1) - assert(columnType.extract(buffer) === values(1)) - assert(buffer.getInt() === 1) + Array(0, 1).foreach { i => + expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) + expectResult(1, "Wrong run length")(buffer.getInt()) + } // ------------- // Tests decoder @@ -123,8 +120,10 @@ class RunLengthEncodingSuite extends FunSuite { val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) - assert(decoder.next() === values(0)) - assert(decoder.next() === values(1)) + Array(0, 1).foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + assert(!decoder.hasNext) } }