From 1c75f1d4983cfedac2a3e7b2c298d5fe9a71009f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 21 Jul 2017 23:17:05 +0900 Subject: [PATCH 01/17] initial commit for batch implementation as @rxin suggested --- .../execution/columnar/ColumnAccessor.scala | 15 + .../sql/execution/columnar/ColumnType.scala | 33 ++ .../CompressibleColumnAccessor.scala | 4 + .../compression/CompressionScheme.scala | 2 + .../compression/compressionSchemes.scala | 330 ++++++++++++++++++ .../compression/BooleanBitSetSuite.scala | 50 +++ .../compression/DictionaryEncodingSuite.scala | 80 ++++- .../compression/IntegralDeltaSuite.scala | 85 ++++- .../compression/RunLengthEncodingSuite.scala | 98 +++++- .../TestCompressibleColumnBuilder.scala | 9 +- .../vectorized/ColumnarBatchSuite.scala | 171 ++++++++- 11 files changed, 864 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index 6241b79d9affc..8c4c8f2bc4c6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -62,6 +62,9 @@ private[columnar] abstract class BasicColumnAccessor[JvmType]( } protected def underlyingBuffer = buffer + + def getByteBuffer: ByteBuffer = + buffer.duplicate.order(ByteOrder.nativeOrder()) } private[columnar] class NullColumnAccessor(buffer: ByteBuffer) @@ -149,4 +152,16 @@ private[columnar] object ColumnAccessor { throw new Exception(s"not support type: $other") } } + + def decompress(columnAccessor: ColumnAccessor, numRows: Int): (ByteBuffer, ByteBuffer) = { + if (columnAccessor.isInstanceOf[NativeColumnAccessor[_]]) { + val nativeAccessor = columnAccessor.asInstanceOf[NativeColumnAccessor[_]] + nativeAccessor.decompress(numRows) + } else { + val buffer = columnAccessor.asInstanceOf[BasicColumnAccessor[_]].getByteBuffer + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + (buffer, nullsBuffer) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 5cfb003e4f150..e9b150fd86095 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -43,6 +43,12 @@ import org.apache.spark.unsafe.types.UTF8String * WARNING: This only works with HeapByteBuffer */ private[columnar] object ByteBufferHelper { + def getShort(buffer: ByteBuffer): Short = { + val pos = buffer.position() + buffer.position(pos + 2) + Platform.getShort(buffer.array(), Platform.BYTE_ARRAY_OFFSET + pos) + } + def getInt(buffer: ByteBuffer): Int = { val pos = buffer.position() buffer.position(pos + 4) @@ -66,6 +72,33 @@ private[columnar] object ByteBufferHelper { buffer.position(pos + 8) Platform.getDouble(buffer.array(), Platform.BYTE_ARRAY_OFFSET + pos) } + + def putShort(buffer: ByteBuffer, value: Short): Unit = { + val pos = buffer.position() + buffer.position(pos + 2) + Platform.putShort(buffer.array(), Platform.BYTE_ARRAY_OFFSET + pos, value) + } + + def putInt(buffer: ByteBuffer, value: Int): Unit = { + val pos = buffer.position() + buffer.position(pos + 4) + Platform.putInt(buffer.array(), Platform.BYTE_ARRAY_OFFSET + pos, value) + } + + def putLong(buffer: ByteBuffer, value: Long): Unit = { + val pos = buffer.position() + buffer.position(pos + 8) + Platform.putLong(buffer.array(), Platform.BYTE_ARRAY_OFFSET + pos, value) + } + + def copyMemory(src: ByteBuffer, dst: ByteBuffer, len: Int): Unit = { + val srcPos = src.position() + val dstPos = dst.position() + src.position(srcPos + len) + dst.position(dstPos + len) + Platform.copyMemory(src.array(), Platform.BYTE_ARRAY_OFFSET + srcPos, + dst.array(), Platform.BYTE_ARRAY_OFFSET + dstPos, len) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala index e1d13ad0e94e5..1183217c18484 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.columnar.compression +import java.nio.ByteBuffer + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnAccessor, NativeColumnAccessor} import org.apache.spark.sql.types.AtomicType @@ -36,4 +38,6 @@ private[columnar] trait CompressibleColumnAccessor[T <: AtomicType] extends Colu override def extractSingle(row: InternalRow, ordinal: Int): Unit = { decoder.next(row, ordinal) } + + def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = decoder.decompress(capacity) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala index 6e4f1c5b80684..7878a49f676ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala @@ -41,6 +41,8 @@ private[columnar] trait Decoder[T <: AtomicType] { def next(row: InternalRow, ordinal: Int): Unit def hasNext: Boolean + + def decompress(capacity: Int): (ByteBuffer, ByteBuffer) } private[columnar] trait CompressionScheme { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index ee99c90a751d9..61bb2614b4a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.ByteBuffer +import java.nio.ByteOrder import scala.collection.mutable @@ -61,6 +62,51 @@ private[columnar] case object PassThrough extends CompressionScheme { } override def hasNext: Boolean = buffer.hasRemaining + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + if (nullCount == 0) { + nullsBuffer.rewind() + (buffer.duplicate().order(ByteOrder.nativeOrder()), nullsBuffer) + } else { + val unitSize = columnType.dataType match { + case _: BooleanType => 1 + case _: ByteType => 1 + case _: ShortType => 2 + case _: IntegerType => 4 + case _: LongType => 8 + case _: FloatType => 4 + case _: DoubleType => 8 + case _ => throw new IllegalStateException("Not supported type in PassThru.") + } + var nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + var pos = 0 + var seenNulls = 0 + val out = ByteBuffer.allocate(capacity * unitSize).order(ByteOrder.nativeOrder()) + while (buffer.hasRemaining) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + ByteBufferHelper.copyMemory(buffer, out, len * unitSize) + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + out.position(out.position + unitSize) + pos += 1 + } + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + } + } } } @@ -169,6 +215,145 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } override def hasNext: Boolean = valueCount < run || buffer.hasRemaining + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 + var pos = 0 + var seenNulls = 0 + var runLocal = 0 + var valueCountLocal = 0 + columnType.dataType match { + case _: BooleanType => + val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) + var currentValueLocal: Boolean = false + while (valueCountLocal < runLocal || buffer.hasRemaining) { + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = buffer.get() == 1 + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + out.put(if (currentValueLocal) 1: Byte else 0: Byte) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 1) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _: ByteType => + val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) + var currentValueLocal: Byte = 0 + while (valueCountLocal < runLocal || buffer.hasRemaining) { + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = buffer.get() + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + out.put(currentValueLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 1) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _: ShortType => + val out = ByteBuffer.allocate(capacity * 2).order(ByteOrder.nativeOrder()) + var currentValueLocal: Short = 0 + while (valueCountLocal < runLocal || buffer.hasRemaining) { + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = buffer.getShort() + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + ByteBufferHelper.putShort(out, currentValueLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 2) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _: IntegerType => + val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) + var currentValueLocal: Int = 0 + while (valueCountLocal < runLocal || buffer.hasRemaining) { + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = buffer.getInt() + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + ByteBufferHelper.putInt(out, currentValueLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 4) + } + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _: LongType => + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) + var currentValueLocal: Long = 0 + while (valueCountLocal < runLocal || buffer.hasRemaining) { + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = buffer.getLong() + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + ByteBufferHelper.putLong(out, currentValueLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 8) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _ => throw new IllegalStateException("Not supported type in RunLengthEncoding.") + } + } } } @@ -278,6 +463,54 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { } override def hasNext: Boolean = buffer.hasRemaining + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 + var pos = 0 + var seenNulls = 0 + columnType.dataType match { + case _: IntegerType => + val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) + while (buffer.hasRemaining) { + if (pos != nextNullIndex) { + val value = dictionary(buffer.getShort()).asInstanceOf[Int] + ByteBufferHelper.putInt(out, value) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 4) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _: LongType => + val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) + while (buffer.hasRemaining) { + if (pos != nextNullIndex) { + val value = dictionary(buffer.getShort()).asInstanceOf[Long] + ByteBufferHelper.putLong(out, value) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 8) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + case _ => throw new IllegalStateException("Not supported type in DictionaryEncoding.") + } + } } } @@ -368,6 +601,42 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { } override def hasNext: Boolean = visited < count + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + val countLocal = count + var currentWordLocal: Long = 0 + var visitedLocal: Int = 0 + val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 + var pos = 0 + var seenNulls = 0 + + while (visitedLocal < countLocal) { + if (pos != nextNullIndex) { + val bit = visitedLocal % BITS_PER_LONG + + visitedLocal += 1 + if (bit == 0) { + currentWordLocal = ByteBufferHelper.getLong(buffer) + } + + out.put(if (((currentWordLocal >> bit) & 1) != 0) 1: Byte else 0: Byte) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 1) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + } } } @@ -448,6 +717,37 @@ private[columnar] case object IntDelta extends CompressionScheme { prev = if (delta > Byte.MinValue) prev + delta else ByteBufferHelper.getInt(buffer) row.setInt(ordinal, prev) } + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + var prevLocal: Int = 0 + val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 + var pos = 0 + var seenNulls = 0 + + while (buffer.hasRemaining) { + if (pos != nextNullIndex) { + val delta = buffer.get + prevLocal = if (delta > Byte.MinValue) { prevLocal + delta } else + { ByteBufferHelper.getInt(buffer) } + val p = out.position + ByteBufferHelper.putInt(out, prevLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 4) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + } } } @@ -528,5 +828,35 @@ private[columnar] case object LongDelta extends CompressionScheme { prev = if (delta > Byte.MinValue) prev + delta else ByteBufferHelper.getLong(buffer) row.setLong(ordinal, prev) } + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + var prevLocal: Long = 0 + val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 + var pos = 0 + var seenNulls = 0 + + while (buffer.hasRemaining) { + if (pos != nextNullIndex) { + val delta = buffer.get() + prevLocal = if (delta > Byte.MinValue) { prevLocal + delta } else + { ByteBufferHelper.getLong(buffer) } + ByteBufferHelper.putLong(out, prevLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + out.position(out.position + 8) + } + pos += 1 + } + out.rewind() + nullsBuffer.rewind() + (out, nullsBuffer) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala index d01bf911e3a77..8dac6778dbd35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala @@ -85,6 +85,36 @@ class BooleanBitSetSuite extends SparkFunSuite { assert(!decoder.hasNext) } + def skeletonForDecompress(count: Int) { + val builder = TestCompressibleColumnBuilder(new NoopColumnStats, BOOLEAN, BooleanBitSet) + val rows = Seq.fill[InternalRow](count)(makeRandomRow(BOOLEAN)) + val values = rows.map(_.getBoolean(0)) + + rows.foreach(builder.appendFrom(_, 0)) + val buffer = builder.build() + + // ---------------- + // Tests decompress + // ---------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + val headerSize = CompressionScheme.columnHeaderSize(buffer) + buffer.position(headerSize) + assertResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) + val (decodeBuffer, nullsBuffer) = decoder.decompress(values.length) + + if (values.nonEmpty) { + values.zipWithIndex.foreach { case (b: Boolean, index: Int) => + assertResult(b, s"Wrong ${index}-th decoded boolean value") { + if (decodeBuffer.get() == 1) true else false + } + } + } + assert(!decodeBuffer.hasRemaining) + } + test(s"$BooleanBitSet: empty") { skeleton(0) } @@ -104,4 +134,24 @@ class BooleanBitSetSuite extends SparkFunSuite { test(s"$BooleanBitSet: multiple words and 1 more bit") { skeleton(BITS_PER_LONG * 2 + 1) } + + test(s"$BooleanBitSet: empty for decompression()") { + skeletonForDecompress(0) + } + + test(s"$BooleanBitSet: less than 1 word for decompression()") { + skeletonForDecompress(BITS_PER_LONG - 1) + } + + test(s"$BooleanBitSet: exactly 1 word for decompression()") { + skeletonForDecompress(BITS_PER_LONG) + } + + test(s"$BooleanBitSet: multiple whole words for decompression()") { + skeletonForDecompress(BITS_PER_LONG * 2) + } + + test(s"$BooleanBitSet: multiple words and 1 more bit for decompression()") { + skeletonForDecompress(BITS_PER_LONG * 2 + 1) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index 67139b13d7882..b1d39feaed66e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -26,13 +26,16 @@ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.AtomicType class DictionaryEncodingSuite extends SparkFunSuite { + val nullValue = -1 testDictionaryEncoding(new IntColumnStats, INT) testDictionaryEncoding(new LongColumnStats, LONG) testDictionaryEncoding(new StringColumnStats, STRING) + testDictionaryEncoding(new StringColumnStats, STRING, false) def testDictionaryEncoding[T <: AtomicType]( - columnStats: ColumnStats, - columnType: NativeColumnType[T]) { + columnStats: ColumnStats, + columnType: NativeColumnType[T], + testDecompress: Boolean = true) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") @@ -113,6 +116,66 @@ class DictionaryEncodingSuite extends SparkFunSuite { } } + def skeletonForDecompress(uniqueValueCount: Int, inputSeq: Seq[Int]) { + if (!testDecompress) return + val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) + val dictValues = stableDistinct(inputSeq) + + val nullRow = new GenericInternalRow(1) + nullRow.setNullAt(0) + inputSeq.foreach { i => + if (i == nullValue) { + builder.appendFrom(nullRow, 0) + } else { + builder.appendFrom(rows(i), 0) + } + } + val buffer = builder.build() + + // ---------------- + // Tests decompress + // ---------------- + // Rewinds, skips column header and 4 more bytes for compression scheme ID + val headerSize = CompressionScheme.columnHeaderSize(buffer) + buffer.position(headerSize) + assertResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val decoder = DictionaryEncoding.decoder(buffer, columnType) + val (decodeBuffer, nullsBuffer) = decoder.decompress(inputSeq.length) + + if (inputSeq.nonEmpty) { + val numNulls = ByteBufferHelper.getInt(nullsBuffer) + var cntNulls = 0 + var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) + inputSeq.zipWithIndex.foreach { case (i: Any, index: Int) => + if (i == nullValue) { + assertResult(index, "Wrong null position") { + nullPos + } + decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) + cntNulls += 1 + if (cntNulls < numNulls) { + nullPos = ByteBufferHelper.getInt(nullsBuffer) + } + } else { + columnType match { + case INT => + assertResult(values(i), s"Wrong ${index}-th decoded int value") { + ByteBufferHelper.getInt(decodeBuffer) + } + case LONG => + assertResult(values(i), s"Wrong ${index}-th decoded long value") { + ByteBufferHelper.getLong(decodeBuffer) + } + case _ => fail("Unsupported type") + } + } + } + } + assert(!decodeBuffer.hasRemaining) + } + test(s"$DictionaryEncoding with $typeName: empty") { skeleton(0, Seq.empty) } @@ -124,5 +187,18 @@ class DictionaryEncodingSuite extends SparkFunSuite { test(s"$DictionaryEncoding with $typeName: dictionary overflow") { skeleton(DictionaryEncoding.MAX_DICT_SIZE + 1, 0 to DictionaryEncoding.MAX_DICT_SIZE) } + + test(s"$DictionaryEncoding with $typeName: empty for decompress()") { + skeletonForDecompress(0, Seq.empty) + } + + test(s"$DictionaryEncoding with $typeName: simple case for decompress()") { + skeletonForDecompress(2, Seq(0, nullValue, 0, nullValue)) + } + + test(s"$DictionaryEncoding with $typeName: dictionary overflow for decompress()") { + skeletonForDecompress(DictionaryEncoding.MAX_DICT_SIZE + 2, + Seq(nullValue) ++ (0 to DictionaryEncoding.MAX_DICT_SIZE - 1) ++ Seq(nullValue)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index 411d31fa0e29b..4a208d3aefaeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -24,13 +24,14 @@ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.IntegralType class IntegralDeltaSuite extends SparkFunSuite { + val nullValue = -1 testIntegralDelta(new IntColumnStats, INT, IntDelta) testIntegralDelta(new LongColumnStats, LONG, LongDelta) def testIntegralDelta[I <: IntegralType]( - columnStats: ColumnStats, - columnType: NativeColumnType[I], - scheme: CompressionScheme) { + columnStats: ColumnStats, + columnType: NativeColumnType[I], + scheme: CompressionScheme) { def skeleton(input: Seq[I#InternalType]) { // ------------- @@ -109,6 +110,61 @@ class IntegralDeltaSuite extends SparkFunSuite { assert(!decoder.hasNext) } + def skeletonForDecompress(input: Seq[I#InternalType]) { + val builder = TestCompressibleColumnBuilder(columnStats, columnType, scheme) + val row = new GenericInternalRow(1) + val nullRow = new GenericInternalRow(1) + nullRow.setNullAt(0) + input.map { value => + if (value == nullValue) { + builder.appendFrom(nullRow, 0) + } else { + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + } + val buffer = builder.build() + + // ---------------- + // Tests decompress + // ---------------- + // Rewinds, skips column header and 4 more bytes for compression scheme ID + val headerSize = CompressionScheme.columnHeaderSize(buffer) + buffer.position(headerSize) + assertResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val decoder = scheme.decoder(buffer, columnType) + val (decodeBuffer, nullsBuffer) = decoder.decompress(input.length) + + if (input.nonEmpty) { + val numNulls = ByteBufferHelper.getInt(nullsBuffer) + var cntNulls = 0 + var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) + input.zipWithIndex.foreach { + case (expected: Any, index: Int) if expected == nullValue => + assertResult(index, "Wrong null position") { + nullPos + } + decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) + cntNulls += 1 + if (cntNulls < numNulls) { + nullPos = ByteBufferHelper.getInt(nullsBuffer) + } + case (expected: Int, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded int value") { + ByteBufferHelper.getInt(decodeBuffer) + } + case (expected: Long, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded long value") { + ByteBufferHelper.getLong(decodeBuffer) + } + case _ => + fail("Unsupported type") + } + } + assert(!decodeBuffer.hasRemaining) + } + test(s"$scheme: empty column") { skeleton(Seq.empty) } @@ -127,5 +183,28 @@ class IntegralDeltaSuite extends SparkFunSuite { val input = Array.fill[Any](10000)(makeRandomValue(columnType)) skeleton(input.map(_.asInstanceOf[I#InternalType])) } + + + test(s"$scheme: empty column for decompress()") { + skeletonForDecompress(Seq.empty) + } + + test(s"$scheme: simple case for decompress()") { + val input = columnType match { + case INT => Seq(2: Int, 1: Int, 2: Int, 130: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, 130: Long) + } + + skeletonForDecompress(input.map(_.asInstanceOf[I#InternalType])) + } + + test(s"$scheme: simple case with null for decompress()") { + val input = columnType match { + case INT => Seq(2: Int, 1: Int, 2: Int, nullValue: Int, 5: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, nullValue: Long, 5: Long) + } + + skeletonForDecompress(input.map(_.asInstanceOf[I#InternalType])) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index dffa9b364ebfe..dea240fb663f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -24,16 +24,18 @@ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.AtomicType class RunLengthEncodingSuite extends SparkFunSuite { + val nullValue = -1 testRunLengthEncoding(new NoopColumnStats, BOOLEAN) testRunLengthEncoding(new ByteColumnStats, BYTE) testRunLengthEncoding(new ShortColumnStats, SHORT) testRunLengthEncoding(new IntColumnStats, INT) testRunLengthEncoding(new LongColumnStats, LONG) - testRunLengthEncoding(new StringColumnStats, STRING) + testRunLengthEncoding(new StringColumnStats, STRING, false) def testRunLengthEncoding[T <: AtomicType]( - columnStats: ColumnStats, - columnType: NativeColumnType[T]) { + columnStats: ColumnStats, + columnType: NativeColumnType[T], + testDecompress: Boolean = true) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") @@ -95,6 +97,80 @@ class RunLengthEncodingSuite extends SparkFunSuite { assert(!decoder.hasNext) } + def skeletonForDecompress(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]) { + if (!testDecompress) return + val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) + val inputSeq = inputRuns.flatMap { case (index, run) => + Seq.fill(run)(index) + } + + val nullRow = new GenericInternalRow(1) + nullRow.setNullAt(0) + inputSeq.foreach { i => + if (i == nullValue) { + builder.appendFrom(nullRow, 0) + } else { + builder.appendFrom(rows(i), 0) + } + } + val buffer = builder.build() + + // ---------------- + // Tests decompress + // ---------------- + // Rewinds, skips column header and 4 more bytes for compression scheme ID + val headerSize = CompressionScheme.columnHeaderSize(buffer) + buffer.position(headerSize) + assertResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val decoder = RunLengthEncoding.decoder(buffer, columnType) + val (decodeBuffer, nullsBuffer) = decoder.decompress(inputSeq.length) + + if (inputSeq.nonEmpty) { + val numNulls = ByteBufferHelper.getInt(nullsBuffer) + var cntNulls = 0 + var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) + inputSeq.zipWithIndex.foreach { + case (expected: Any, index: Int) if expected == nullValue => + assertResult(index, "Wrong null position") { + nullPos + } + decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) + cntNulls += 1 + if (cntNulls < numNulls) { + nullPos = ByteBufferHelper.getInt(nullsBuffer) + } + case (i: Int, index: Int) => + columnType match { + case BOOLEAN => + assertResult(values(i), s"Wrong ${index}-th decoded boolean value") { + if (decodeBuffer.get() == 1) true else false + } + case BYTE => + assertResult(values(i), s"Wrong ${index}-th decoded byte value") { + decodeBuffer.get() + } + case SHORT => + assertResult(values(i), s"Wrong ${index}-th decoded short value") { + ByteBufferHelper.getShort(decodeBuffer) + } + case INT => + assertResult(values(i), s"Wrong ${index}-th decoded int value") { + ByteBufferHelper.getInt(decodeBuffer) + } + case LONG => + assertResult(values(i), s"Wrong ${index}-th decoded long value") { + ByteBufferHelper.getLong(decodeBuffer) + } + case _ => fail("Unsupported type") + } + case _ => fail("Unsupported type") + } + } + assert(!decodeBuffer.hasRemaining) + } + test(s"$RunLengthEncoding with $typeName: empty column") { skeleton(0, Seq.empty) } @@ -110,5 +186,21 @@ class RunLengthEncodingSuite extends SparkFunSuite { test(s"$RunLengthEncoding with $typeName: single long run") { skeleton(1, Seq(0 -> 1000)) } + + test(s"$RunLengthEncoding with $typeName: empty column for decompress()") { + skeletonForDecompress(0, Seq.empty) + } + + test(s"$RunLengthEncoding with $typeName: simple case for decompress()") { + skeletonForDecompress(2, Seq(0 -> 2, 1 -> 2)) + } + + test(s"$RunLengthEncoding with $typeName: single long run for decompress()") { + skeletonForDecompress(1, Seq(0 -> 1000)) + } + + test(s"$RunLengthEncoding with $typeName: single case with null for decompress()") { + skeletonForDecompress(2, Seq(0 -> 2, nullValue -> 2)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala index 5e078f251375a..310cb0be5f5a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.sql.execution.columnar._ -import org.apache.spark.sql.types.AtomicType +import org.apache.spark.sql.types.{AtomicType, DataType} class TestCompressibleColumnBuilder[T <: AtomicType]( override val columnStats: ColumnStats, @@ -42,3 +42,10 @@ object TestCompressibleColumnBuilder { builder } } + +object ColumnBuilderHelper { + def apply( + dataType: DataType, batchSize: Int, name: String, useCompression: Boolean): ColumnBuilder = { + ColumnBuilder(dataType, batchSize, name, useCompression) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 983eb103682c1..32c178debf455 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -29,12 +29,14 @@ import org.apache.arrow.vector.NullableIntVector import org.apache.spark.SparkFunSuite import org.apache.spark.memory.MemoryMode +import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.arrow.ArrowUtils +import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.execution.columnar.compression.ColumnBuilderHelper import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class ColumnarBatchSuite extends SparkFunSuite { @@ -413,7 +415,7 @@ class ColumnarBatchSuite extends SparkFunSuite { reference.zipWithIndex.foreach { v => assert(v._1 == column.getLong(v._2), "idx=" + v._2 + - " Seed = " + seed + " MemMode=" + memMode) + " Seed = " + seed + " MemMode=" + memMode) if (memMode == MemoryMode.OFF_HEAP) { val addr = column.valuesNativeAddress() assert(v._1 == Platform.getLong(null, addr + 8 * v._2)) @@ -1120,7 +1122,7 @@ class ColumnarBatchSuite extends SparkFunSuite { } batch.close() } - }} + }} /** * This test generates a random schema data, serializes it to column batches and verifies the @@ -1293,4 +1295,165 @@ class ColumnarBatchSuite extends SparkFunSuite { batch.close() allocator.close() } + + test("CachedBatch boolean Apis") { + val dataType = BooleanType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setBoolean(0, i % 2 == 0) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getBoolean(i) == (i % 2 == 0)) + } + column.close + } + + test("CachedBatch byte Apis") { + val dataType = ByteType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setByte(0, i.toByte) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getByte(i) == i) + } + column.close + } + + test("CachedBatch short Apis") { + val dataType = ShortType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setShort(0, i.toShort) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getShort(i) == i) + } + column.close + } + + test("CachedBatch int Apis") { + val dataType = IntegerType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setInt(0, i) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getInt(i) == i) + } + column.close + } + + test("CachedBatch long Apis") { + val dataType = LongType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setLong(0, i.toLong) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getLong(i) == i.toLong) + } + column.close + } + + test("CachedBatch float Apis") { + val dataType = FloatType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setFloat(0, i.toFloat) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getFloat(i) == i.toFloat) + } + column.close + } + + test("CachedBatch double Apis") { + val dataType = DoubleType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setDouble(0, i.toDouble) + columnBuilder.appendFrom(row, 0) + } + + val column = new CachedBatchColumnVector( + JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + + assert(column.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(column.isNullAt(i) == false) + assert(column.getDouble(i) == i.toDouble) + } + column.close + } } From 73e5aeef939f97dbb314a0689eaa76521a774c07 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 21 Jul 2017 23:41:42 +0900 Subject: [PATCH 02/17] add missing files --- .../vectorized/CachedBatchColumnVector.java | 239 ++++++++++++++++++ .../PassThroughEncodingSuite.scala | 196 ++++++++++++++ 2 files changed, 435 insertions(+) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java new file mode 100644 index 0000000000000..7cf2de6d40051 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java @@ -0,0 +1,239 @@ +/* + * 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.execution.vectorized; + +import java.nio.ByteBuffer; + +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder; +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; +import org.apache.spark.sql.execution.columnar.*; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.types.UTF8String; +import scala.Tuple2; + +/** + * A column vector backed by data compressed thru ColumnAccessor + * this is a wrapper to read compressed data for table cache + */ +public final class CachedBatchColumnVector extends ReadOnlyColumnVector { + + // accessor for a column + private ColumnAccessor columnAccessor; + + // Array for decompressed null information in a column + private byte[] nulls; + + // Array for decompressed data information in a column + private byte[] data; + private long offset; + + + public CachedBatchColumnVector(byte[] buffer, int numRows, DataType type) { + super(numRows, type, MemoryMode.ON_HEAP); + initialize(buffer, type); + } + + @Override + public long valuesNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + @Override + public long nullsNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + + @Override + public void close() { + } + + // + // APIs dealing with nulls + // + + @Override + public boolean isNullAt(int rowId) { + return nulls[rowId] == 1; + } + + // + // APIs dealing with Booleans + // + + @Override + public boolean getBoolean(int rowId) { + return Platform.getBoolean(data, offset + rowId); + } + + @Override + public boolean[] getBooleans(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + // + + // + // APIs dealing with Bytes + // + + @Override + public byte getByte(int rowId) { + return Platform.getByte(data, offset + rowId); + } + + @Override + public byte[] getBytes(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with Shorts + // + + @Override + public short getShort(int rowId) { + return Platform.getShort(data, offset + rowId * 2); + } + + @Override + public short[] getShorts(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with Ints + // + + @Override + public int getInt(int rowId) { + return Platform.getInt(data, offset + rowId * 4); + } + + @Override + public int[] getInts(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + public int getDictId(int rowId) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with Longs + // + + @Override + public long getLong(int rowId) { + return Platform.getLong(data, offset + rowId * 8); + } + + @Override + public long[] getLongs(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with floats + // + + @Override + public float getFloat(int rowId) { + return Platform.getFloat(data, offset + rowId * 4); + } + + @Override + public float[] getFloats(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with doubles + // + + @Override + public double getDouble(int rowId) { + return Platform.getDouble(data, offset + rowId * 8); + } + + @Override + public double[] getDoubles(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with Arrays + // + + @Override + public int getArrayLength(int rowId) { + throw new UnsupportedOperationException(); + } + @Override + public int getArrayOffset(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public void loadBytes(ColumnVector.Array array) { + throw new UnsupportedOperationException(); + } + + // + // APIs dealing with Byte Arrays + // + + public final UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + private void initialize(byte[] buffer, DataType type) { + ByteBuffer byteBuffer = ByteBuffer.wrap(buffer); + columnAccessor = ColumnAccessor$.MODULE$.apply(type, byteBuffer); + + if (type instanceof StringType) { + throw new UnsupportedOperationException(); + } else if (type instanceof ArrayType) { + throw new UnsupportedOperationException(); + } else if (type instanceof BinaryType) { + throw new UnsupportedOperationException(); + } else if (type instanceof StructType) { + throw new UnsupportedOperationException(); + } else if (type instanceof MapType) { + throw new UnsupportedOperationException(); + } else if (type instanceof DecimalType && ((DecimalType) type).precision() > Decimal.MAX_LONG_DIGITS()) { + throw new UnsupportedOperationException(); + } + + Tuple2 buffers = ColumnAccessor$.MODULE$.decompress(columnAccessor, capacity); + ByteBuffer dataBuffer = (ByteBuffer)(buffers._1()); + ByteBuffer nullsBuffer = (ByteBuffer)(buffers._2()); + + int numNulls = ByteBufferHelper.getInt(nullsBuffer); + if (numNulls > 0) { + nulls = new byte[capacity]; + anyNullsSet = true; + } + for (int i = 0; i < numNulls; i++) { + int cordinal = ByteBufferHelper.getInt(nullsBuffer); + nulls[cordinal] = (byte)1; + } + data = dataBuffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + dataBuffer.position(); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala new file mode 100644 index 0000000000000..ad48e91f72934 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.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.execution.columnar.compression + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.{AtomicType, IntegralType} + +class PassThroughSuite extends SparkFunSuite { + val nullValue = -1 + testPassThrough(new ByteColumnStats, BYTE) + testPassThrough(new ShortColumnStats, SHORT) + testPassThrough(new IntColumnStats, INT) + testPassThrough(new LongColumnStats, LONG) + testPassThrough(new FloatColumnStats, FLOAT) + testPassThrough(new DoubleColumnStats, DOUBLE) + + def testPassThrough[T <: AtomicType]( + columnStats: ColumnStats, + columnType: NativeColumnType[T]) { + + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + def skeleton(input: Seq[T#InternalType]) { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) + + input.map { value => + val row = new GenericInternalRow(1) + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + + val buffer = builder.build() + // Column type ID + null count + null positions + val headerSize = CompressionScheme.columnHeaderSize(buffer) + + // Compression scheme ID + compressed contents + val compressedSize = 4 + input.size * columnType.defaultSize + + // 4 extra bytes for compression scheme type ID + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + buffer.position(headerSize) + assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + if (input.nonEmpty) { + input.foreach { value => + assertResult(value, "Wrong value")(columnType.extract(buffer)) + } + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = PassThrough.decoder(buffer, columnType) + val mutableRow = new GenericInternalRow(1) + + if (input.nonEmpty) { + input.foreach{ + assert(decoder.hasNext) + assertResult(_, "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } + } + } + assert(!decoder.hasNext) + } + + def skeletonForDecompress(input: Seq[T#InternalType]) { + val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) + val row = new GenericInternalRow(1) + val nullRow = new GenericInternalRow(1) + nullRow.setNullAt(0) + input.map { value => + if (value == nullValue) { + builder.appendFrom(nullRow, 0) + } else { + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + } + val buffer = builder.build() + + // ---------------- + // Tests decompress + // ---------------- + // Rewinds, skips column header and 4 more bytes for compression scheme ID + val headerSize = CompressionScheme.columnHeaderSize(buffer) + buffer.position(headerSize) + assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val decoder = PassThrough.decoder(buffer, columnType) + val (decodeBuffer, nullsBuffer) = decoder.decompress(input.length) + + if (input.nonEmpty) { + val numNulls = ByteBufferHelper.getInt(nullsBuffer) + var cntNulls = 0 + var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) + input.zipWithIndex.foreach { + case (expected: Any, index: Int) if expected == nullValue => + assertResult(index, "Wrong null position") { + nullPos + } + decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) + cntNulls += 1 + if (cntNulls < numNulls) { + nullPos = ByteBufferHelper.getInt(nullsBuffer) + } + case (expected: Byte, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded byte value") { + decodeBuffer.get() + } + case (expected: Short, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded short value") { + ByteBufferHelper.getShort(decodeBuffer) + } + case (expected: Int, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded int value") { + ByteBufferHelper.getInt(decodeBuffer) + } + case (expected: Long, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded long value") { + ByteBufferHelper.getLong(decodeBuffer) + } + case (expected: Float, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded float value") { + ByteBufferHelper.getFloat(decodeBuffer) + } + case (expected: Double, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded double value") { + ByteBufferHelper.getDouble(decodeBuffer) + } + case _ => fail("Unsupported type") + } + } + assert(!decodeBuffer.hasRemaining) + } + + test(s"$PassThrough with $typeName: empty column") { + skeleton(Seq.empty) + } + + test(s"$PassThrough with $typeName: long random series") { + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeleton(input.map(_.asInstanceOf[T#InternalType])) + } + + test(s"$PassThrough with $typeName: empty column for decompress()") { + skeletonForDecompress(Seq.empty) + } + + test(s"$PassThrough with $typeName: long random series for decompress()") { + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) + } + + test(s"$PassThrough with $typeName: simple case with null for decompress()") { + val input = columnType match { + case BYTE => Seq(2: Byte, 1: Byte, 2: Byte, nullValue.toByte: Byte, 5: Byte) + case SHORT => Seq(2: Short, 1: Short, 2: Short, nullValue.toShort: Short, 5: Short) + case INT => Seq(2: Int, 1: Int, 2: Int, nullValue: Int, 5: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, nullValue: Long, 5: Long) + case FLOAT => Seq(2: Float, 1: Float, 2: Float, nullValue: Float, 5: Float) + case DOUBLE => Seq(2: Double, 1: Double, 2: Double, nullValue: Double, 5: Double) + } + + skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) + } + } +} \ No newline at end of file From 591a358501c52e34d03b3c02c0681dbfe1ff7252 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 21 Jul 2017 23:51:07 +0900 Subject: [PATCH 03/17] fix scala style error --- .../columnar/compression/PassThroughEncodingSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index ad48e91f72934..437750fd2364d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -193,4 +193,4 @@ class PassThroughSuite extends SparkFunSuite { skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) } } -} \ No newline at end of file +} From eb879ac560582f76f3eceeadbfec8d764d5c74f4 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 22 Jul 2017 02:04:37 +0900 Subject: [PATCH 04/17] fix test failure of DictionaryEncodingSuite --- .../columnar/compression/DictionaryEncodingSuite.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index b1d39feaed66e..a8670828a1cd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -29,13 +29,12 @@ class DictionaryEncodingSuite extends SparkFunSuite { val nullValue = -1 testDictionaryEncoding(new IntColumnStats, INT) testDictionaryEncoding(new LongColumnStats, LONG) - testDictionaryEncoding(new StringColumnStats, STRING) testDictionaryEncoding(new StringColumnStats, STRING, false) def testDictionaryEncoding[T <: AtomicType]( - columnStats: ColumnStats, - columnType: NativeColumnType[T], - testDecompress: Boolean = true) { + columnStats: ColumnStats, + columnType: NativeColumnType[T], + testDecompress: Boolean = true) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") From 10951d2f307fe84b36799b0c3dcd2a0b574871ec Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 22 Aug 2017 06:34:01 +0100 Subject: [PATCH 05/17] add new APIs for adding values from a byte array --- .../vectorized/CachedBatchColumnVector.java | 239 ------------------ .../vectorized/OffHeapColumnVector.java | 18 ++ .../vectorized/OnHeapColumnVector.java | 18 ++ 3 files changed, 36 insertions(+), 239 deletions(-) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java deleted file mode 100644 index 7cf2de6d40051..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/CachedBatchColumnVector.java +++ /dev/null @@ -1,239 +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.execution.vectorized; - -import java.nio.ByteBuffer; - -import org.apache.spark.memory.MemoryMode; -import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder; -import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; -import org.apache.spark.sql.execution.columnar.*; -import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.types.UTF8String; -import scala.Tuple2; - -/** - * A column vector backed by data compressed thru ColumnAccessor - * this is a wrapper to read compressed data for table cache - */ -public final class CachedBatchColumnVector extends ReadOnlyColumnVector { - - // accessor for a column - private ColumnAccessor columnAccessor; - - // Array for decompressed null information in a column - private byte[] nulls; - - // Array for decompressed data information in a column - private byte[] data; - private long offset; - - - public CachedBatchColumnVector(byte[] buffer, int numRows, DataType type) { - super(numRows, type, MemoryMode.ON_HEAP); - initialize(buffer, type); - } - - @Override - public long valuesNativeAddress() { - throw new RuntimeException("Cannot get native address for on heap column"); - } - @Override - public long nullsNativeAddress() { - throw new RuntimeException("Cannot get native address for on heap column"); - } - - @Override - public void close() { - } - - // - // APIs dealing with nulls - // - - @Override - public boolean isNullAt(int rowId) { - return nulls[rowId] == 1; - } - - // - // APIs dealing with Booleans - // - - @Override - public boolean getBoolean(int rowId) { - return Platform.getBoolean(data, offset + rowId); - } - - @Override - public boolean[] getBooleans(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - // - - // - // APIs dealing with Bytes - // - - @Override - public byte getByte(int rowId) { - return Platform.getByte(data, offset + rowId); - } - - @Override - public byte[] getBytes(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with Shorts - // - - @Override - public short getShort(int rowId) { - return Platform.getShort(data, offset + rowId * 2); - } - - @Override - public short[] getShorts(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with Ints - // - - @Override - public int getInt(int rowId) { - return Platform.getInt(data, offset + rowId * 4); - } - - @Override - public int[] getInts(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - public int getDictId(int rowId) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with Longs - // - - @Override - public long getLong(int rowId) { - return Platform.getLong(data, offset + rowId * 8); - } - - @Override - public long[] getLongs(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with floats - // - - @Override - public float getFloat(int rowId) { - return Platform.getFloat(data, offset + rowId * 4); - } - - @Override - public float[] getFloats(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with doubles - // - - @Override - public double getDouble(int rowId) { - return Platform.getDouble(data, offset + rowId * 8); - } - - @Override - public double[] getDoubles(int rowId, int count) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with Arrays - // - - @Override - public int getArrayLength(int rowId) { - throw new UnsupportedOperationException(); - } - @Override - public int getArrayOffset(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public void loadBytes(ColumnVector.Array array) { - throw new UnsupportedOperationException(); - } - - // - // APIs dealing with Byte Arrays - // - - public final UTF8String getUTF8String(int rowId) { - throw new UnsupportedOperationException(); - } - - private void initialize(byte[] buffer, DataType type) { - ByteBuffer byteBuffer = ByteBuffer.wrap(buffer); - columnAccessor = ColumnAccessor$.MODULE$.apply(type, byteBuffer); - - if (type instanceof StringType) { - throw new UnsupportedOperationException(); - } else if (type instanceof ArrayType) { - throw new UnsupportedOperationException(); - } else if (type instanceof BinaryType) { - throw new UnsupportedOperationException(); - } else if (type instanceof StructType) { - throw new UnsupportedOperationException(); - } else if (type instanceof MapType) { - throw new UnsupportedOperationException(); - } else if (type instanceof DecimalType && ((DecimalType) type).precision() > Decimal.MAX_LONG_DIGITS()) { - throw new UnsupportedOperationException(); - } - - Tuple2 buffers = ColumnAccessor$.MODULE$.decompress(columnAccessor, capacity); - ByteBuffer dataBuffer = (ByteBuffer)(buffers._1()); - ByteBuffer nullsBuffer = (ByteBuffer)(buffers._2()); - - int numNulls = ByteBufferHelper.getInt(nullsBuffer); - if (numNulls > 0) { - nulls = new byte[capacity]; - anyNullsSet = true; - } - for (int i = 0; i < numNulls; i++) { - int cordinal = ByteBufferHelper.getInt(nullsBuffer); - nulls[cordinal] = (byte)1; - } - data = dataBuffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + dataBuffer.position(); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 8cbc895506d91..a7522ebf5821a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -228,6 +228,12 @@ public void putShorts(int rowId, int count, short[] src, int srcIndex) { null, data + 2 * rowId, count * 2); } + @Override + public void putShorts(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 2, count * 2); + } + @Override public short getShort(int rowId) { if (dictionary == null) { @@ -268,6 +274,12 @@ public void putInts(int rowId, int count, int[] src, int srcIndex) { null, data + 4 * rowId, count * 4); } + @Override + public void putInts(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 4, count * 4); + } + @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { @@ -334,6 +346,12 @@ public void putLongs(int rowId, int count, long[] src, int srcIndex) { null, data + 8 * rowId, count * 8); } + @Override + public void putLongs(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 8, count * 8); + } + @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { if (!bigEndianPlatform) { diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index 2725a29eeabe8..166a39e0fabd9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -233,6 +233,12 @@ public void putShorts(int rowId, int count, short[] src, int srcIndex) { System.arraycopy(src, srcIndex, shortData, rowId, count); } + @Override + public void putShorts(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, shortData, + Platform.SHORT_ARRAY_OFFSET + rowId * 2, count * 2); + } + @Override public short getShort(int rowId) { if (dictionary == null) { @@ -272,6 +278,12 @@ public void putInts(int rowId, int count, int[] src, int srcIndex) { System.arraycopy(src, srcIndex, intData, rowId, count); } + @Override + public void putInts(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, intData, + Platform.INT_ARRAY_OFFSET + rowId * 4, count * 4); + } + @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; @@ -332,6 +344,12 @@ public void putLongs(int rowId, int count, long[] src, int srcIndex) { System.arraycopy(src, srcIndex, longData, rowId, count); } + @Override + public void putLongs(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, longData, + Platform.LONG_ARRAY_OFFSET + rowId * 8, count * 8); + } + @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; From 5efbd2eb556fcdbce6b909dbc62f5d697f68d5e9 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 22 Aug 2017 07:10:53 +0100 Subject: [PATCH 06/17] Use ColumnVector for ColumnAccessor --- .../execution/columnar/ColumnAccessor.scala | 19 +- .../CompressibleColumnAccessor.scala | 4 +- .../compression/CompressionScheme.scala | 3 +- .../compression/compressionSchemes.scala | 293 +++++++++++------- .../compression/BooleanBitSetSuite.scala | 9 +- .../compression/DictionaryEncodingSuite.scala | 23 +- .../compression/IntegralDeltaSuite.scala | 23 +- .../PassThroughEncodingSuite.scala | 31 +- .../compression/RunLengthEncodingSuite.scala | 29 +- .../vectorized/ColumnarBatchSuite.scala | 36 ++- 10 files changed, 271 insertions(+), 199 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index 8c4c8f2bc4c6c..53e7687a3fb96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -24,6 +24,7 @@ import scala.annotation.tailrec import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeMapData, UnsafeRow} import org.apache.spark.sql.execution.columnar.compression.CompressibleColumnAccessor +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ /** @@ -125,7 +126,7 @@ private[columnar] class MapColumnAccessor(buffer: ByteBuffer, dataType: MapType) extends BasicColumnAccessor[UnsafeMapData](buffer, MAP(dataType)) with NullableColumnAccessor -private[columnar] object ColumnAccessor { +private[sql] object ColumnAccessor { @tailrec def apply(dataType: DataType, buffer: ByteBuffer): ColumnAccessor = { val buf = buffer.order(ByteOrder.nativeOrder) @@ -153,15 +154,21 @@ private[columnar] object ColumnAccessor { } } - def decompress(columnAccessor: ColumnAccessor, numRows: Int): (ByteBuffer, ByteBuffer) = { + def decompress(columnAccessor: ColumnAccessor, columnVector: ColumnVector, numRows: Int): Unit = { if (columnAccessor.isInstanceOf[NativeColumnAccessor[_]]) { val nativeAccessor = columnAccessor.asInstanceOf[NativeColumnAccessor[_]] - nativeAccessor.decompress(numRows) + nativeAccessor.decompress(columnVector, numRows) } else { - val buffer = columnAccessor.asInstanceOf[BasicColumnAccessor[_]].getByteBuffer - val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + val dataBuffer = columnAccessor.asInstanceOf[BasicColumnAccessor[_]].getByteBuffer + val nullsBuffer = dataBuffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() - (buffer, nullsBuffer) + + val numNulls = ByteBufferHelper.getInt(nullsBuffer) + for (i <- 0 until numNulls) { + val cordinal = ByteBufferHelper.getInt(nullsBuffer) + columnVector.putNull(cordinal) + } + throw new RuntimeException("Not support non-primitive type now") } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala index 1183217c18484..856a4b5dc099a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnAccessor, NativeColumnAccessor} +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types.AtomicType private[columnar] trait CompressibleColumnAccessor[T <: AtomicType] extends ColumnAccessor { @@ -39,5 +40,6 @@ private[columnar] trait CompressibleColumnAccessor[T <: AtomicType] extends Colu decoder.next(row, ordinal) } - def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = decoder.decompress(capacity) + def decompress(columnVector: ColumnVector, capacity: Int): Unit = + decoder.decompress(columnVector, capacity) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala index 7878a49f676ec..c471694a1f379 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala @@ -21,6 +21,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnType, NativeColumnType} +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types.AtomicType private[columnar] trait Encoder[T <: AtomicType] { @@ -42,7 +43,7 @@ private[columnar] trait Decoder[T <: AtomicType] { def hasNext: Boolean - def decompress(capacity: Int): (ByteBuffer, ByteBuffer) + def decompress(columnVector: ColumnVector, capacity: Int): Unit } private[columnar] trait CompressionScheme { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 61bb2614b4a1e..59ed0e4f7e086 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -25,6 +25,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ @@ -63,48 +64,159 @@ private[columnar] case object PassThrough extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) - if (nullCount == 0) { - nullsBuffer.rewind() - (buffer.duplicate().order(ByteOrder.nativeOrder()), nullsBuffer) - } else { - val unitSize = columnType.dataType match { - case _: BooleanType => 1 - case _: ByteType => 1 - case _: ShortType => 2 - case _: IntegerType => 4 - case _: LongType => 8 - case _: FloatType => 4 - case _: DoubleType => 8 - case _ => throw new IllegalStateException("Not supported type in PassThru.") - } - var nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - var pos = 0 - var seenNulls = 0 - val out = ByteBuffer.allocate(capacity * unitSize).order(ByteOrder.nativeOrder()) - while (buffer.hasRemaining) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - ByteBufferHelper.copyMemory(buffer, out, len * unitSize) - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) + var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else capacity + var pos = 0 + var seenNulls = 0 + val srcArray = buffer.array + var bufferPos = buffer.position + columnType.dataType match { + case _: BooleanType => + val unitSize = 1 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + for (i <- 0 until len) { + val value = buffer.get(bufferPos + i) != 0 + columnVector.putBoolean(pos + i, value) + } + bufferPos += len + pos += len } else { - capacity + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + case _: ByteType => + val unitSize = 1 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + columnVector.putBytes(pos, len, srcArray, bufferPos) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + case _: ShortType => + val unitSize = 2 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + columnVector.putShorts(pos, len, srcArray, bufferPos) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + case _: IntegerType => + val unitSize = 4 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + columnVector.putInts(pos, len, srcArray, bufferPos) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + case _: LongType => + val unitSize = 8 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + columnVector.putLongs(pos, len, srcArray, bufferPos) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + case _: FloatType => + val unitSize = 4 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + columnVector.putFloats(pos, len, srcArray, bufferPos) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + case _: DoubleType => + val unitSize = 8 + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + columnVector.putDoubles(pos, len, srcArray, bufferPos) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 } - out.position(out.position + unitSize) - pos += 1 } - } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) } } } @@ -216,7 +328,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { override def hasNext: Boolean = valueCount < run || buffer.hasRemaining - override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -227,9 +339,8 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { var valueCountLocal = 0 columnType.dataType match { case _: BooleanType => - val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) var currentValueLocal: Boolean = false - while (valueCountLocal < runLocal || buffer.hasRemaining) { + while (valueCountLocal < runLocal || (pos < capacity)) { if (pos != nextNullIndex) { if (valueCountLocal == runLocal) { currentValueLocal = buffer.get() == 1 @@ -238,23 +349,19 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCountLocal += 1 } - out.put(if (currentValueLocal) 1: Byte else 0: Byte) + columnVector.putBoolean(pos, currentValueLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 1) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _: ByteType => - val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) var currentValueLocal: Byte = 0 - while (valueCountLocal < runLocal || buffer.hasRemaining) { + while (valueCountLocal < runLocal || (pos < capacity)) { if (pos != nextNullIndex) { if (valueCountLocal == runLocal) { currentValueLocal = buffer.get() @@ -263,23 +370,19 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCountLocal += 1 } - out.put(currentValueLocal) + columnVector.putByte(pos, currentValueLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 1) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _: ShortType => - val out = ByteBuffer.allocate(capacity * 2).order(ByteOrder.nativeOrder()) var currentValueLocal: Short = 0 - while (valueCountLocal < runLocal || buffer.hasRemaining) { + while (valueCountLocal < runLocal || (pos < capacity)) { if (pos != nextNullIndex) { if (valueCountLocal == runLocal) { currentValueLocal = buffer.getShort() @@ -288,23 +391,19 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCountLocal += 1 } - ByteBufferHelper.putShort(out, currentValueLocal) + columnVector.putShort(pos, currentValueLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 2) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _: IntegerType => - val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) var currentValueLocal: Int = 0 - while (valueCountLocal < runLocal || buffer.hasRemaining) { + while (valueCountLocal < runLocal || (pos < capacity)) { if (pos != nextNullIndex) { if (valueCountLocal == runLocal) { currentValueLocal = buffer.getInt() @@ -313,23 +412,19 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCountLocal += 1 } - ByteBufferHelper.putInt(out, currentValueLocal) + columnVector.putInt(pos, currentValueLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 4) + columnVector.putNull(pos) } } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _: LongType => val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) - val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) var currentValueLocal: Long = 0 - while (valueCountLocal < runLocal || buffer.hasRemaining) { + while (valueCountLocal < runLocal || (pos < capacity)) { if (pos != nextNullIndex) { if (valueCountLocal == runLocal) { currentValueLocal = buffer.getLong() @@ -338,19 +433,16 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCountLocal += 1 } - ByteBufferHelper.putLong(out, currentValueLocal) + columnVector.putLong(pos, currentValueLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 8) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _ => throw new IllegalStateException("Not supported type in RunLengthEncoding.") } } @@ -464,7 +556,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -473,41 +565,33 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { var seenNulls = 0 columnType.dataType match { case _: IntegerType => - val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) - while (buffer.hasRemaining) { + while (pos < capacity) { if (pos != nextNullIndex) { val value = dictionary(buffer.getShort()).asInstanceOf[Int] - ByteBufferHelper.putInt(out, value) + columnVector.putInt(pos, value) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 4) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _: LongType => - val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) - while (buffer.hasRemaining) { + while (pos < capacity) { if (pos != nextNullIndex) { val value = dictionary(buffer.getShort()).asInstanceOf[Long] - ByteBufferHelper.putLong(out, value) + columnVector.putLong(pos, value) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 8) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) case _ => throw new IllegalStateException("Not supported type in DictionaryEncoding.") } } @@ -602,11 +686,10 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { override def hasNext: Boolean = visited < count - override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { val countLocal = count var currentWordLocal: Long = 0 var visitedLocal: Int = 0 - val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -623,19 +706,16 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { currentWordLocal = ByteBufferHelper.getLong(buffer) } - out.put(if (((currentWordLocal >> bit) & 1) != 0) 1: Byte else 0: Byte) + columnVector.putBoolean(pos, ((currentWordLocal >> bit) & 1) != 0) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 1) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) } } } @@ -718,9 +798,8 @@ private[columnar] case object IntDelta extends CompressionScheme { row.setInt(ordinal, prev) } - override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { var prevLocal: Int = 0 - val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -728,25 +807,21 @@ private[columnar] case object IntDelta extends CompressionScheme { var pos = 0 var seenNulls = 0 - while (buffer.hasRemaining) { + while (pos < capacity) { if (pos != nextNullIndex) { val delta = buffer.get prevLocal = if (delta > Byte.MinValue) { prevLocal + delta } else { ByteBufferHelper.getInt(buffer) } - val p = out.position - ByteBufferHelper.putInt(out, prevLocal) + columnVector.putInt(pos, prevLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 4) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) } } } @@ -829,9 +904,8 @@ private[columnar] case object LongDelta extends CompressionScheme { row.setLong(ordinal, prev) } - override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { var prevLocal: Long = 0 - val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -839,24 +913,21 @@ private[columnar] case object LongDelta extends CompressionScheme { var pos = 0 var seenNulls = 0 - while (buffer.hasRemaining) { + while (pos < capacity) { if (pos != nextNullIndex) { val delta = buffer.get() prevLocal = if (delta > Byte.MinValue) { prevLocal + delta } else { ByteBufferHelper.getLong(buffer) } - ByteBufferHelper.putLong(out, prevLocal) + columnVector.putLong(pos, prevLocal) } else { seenNulls += 1 if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 8) + columnVector.putNull(pos) } pos += 1 } - out.rewind() - nullsBuffer.rewind() - (out, nullsBuffer) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala index 8dac6778dbd35..52ab1aa57d900 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala @@ -18,10 +18,13 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar.{BOOLEAN, NoopColumnStats} import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.types.BooleanType class BooleanBitSetSuite extends SparkFunSuite { import BooleanBitSet._ @@ -103,16 +106,16 @@ class BooleanBitSetSuite extends SparkFunSuite { assertResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) - val (decodeBuffer, nullsBuffer) = decoder.decompress(values.length) + val columnVector = ColumnVector.allocate(values.length, BooleanType, MemoryMode.ON_HEAP) + decoder.decompress(columnVector, values.length) if (values.nonEmpty) { values.zipWithIndex.foreach { case (b: Boolean, index: Int) => assertResult(b, s"Wrong ${index}-th decoded boolean value") { - if (decodeBuffer.get() == 1) true else false + columnVector.getBoolean(index) } } } - assert(!decodeBuffer.hasRemaining) } test(s"$BooleanBitSet: empty") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index a8670828a1cd6..e99f1c235d340 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -20,9 +20,11 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.ByteBuffer import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types.AtomicType class DictionaryEncodingSuite extends SparkFunSuite { @@ -141,38 +143,31 @@ class DictionaryEncodingSuite extends SparkFunSuite { assertResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = DictionaryEncoding.decoder(buffer, columnType) - val (decodeBuffer, nullsBuffer) = decoder.decompress(inputSeq.length) + val columnVector = ColumnVector.allocate(inputSeq.length, columnType.dataType, + MemoryMode.ON_HEAP) + decoder.decompress(columnVector, inputSeq.length) if (inputSeq.nonEmpty) { - val numNulls = ByteBufferHelper.getInt(nullsBuffer) - var cntNulls = 0 - var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) inputSeq.zipWithIndex.foreach { case (i: Any, index: Int) => if (i == nullValue) { - assertResult(index, "Wrong null position") { - nullPos - } - decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) - cntNulls += 1 - if (cntNulls < numNulls) { - nullPos = ByteBufferHelper.getInt(nullsBuffer) + assertResult(true, s"Wrong null ${index}-th position") { + columnVector.isNullAt(index) } } else { columnType match { case INT => assertResult(values(i), s"Wrong ${index}-th decoded int value") { - ByteBufferHelper.getInt(decodeBuffer) + columnVector.getInt(index) } case LONG => assertResult(values(i), s"Wrong ${index}-th decoded long value") { - ByteBufferHelper.getLong(decodeBuffer) + columnVector.getLong(index) } case _ => fail("Unsupported type") } } } } - assert(!decodeBuffer.hasRemaining) } test(s"$DictionaryEncoding with $typeName: empty") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index 4a208d3aefaeb..c3d1eba019cd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types.IntegralType class IntegralDeltaSuite extends SparkFunSuite { @@ -134,35 +136,28 @@ class IntegralDeltaSuite extends SparkFunSuite { assertResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = scheme.decoder(buffer, columnType) - val (decodeBuffer, nullsBuffer) = decoder.decompress(input.length) + val columnVector = ColumnVector.allocate(input.length, columnType.dataType, + MemoryMode.ON_HEAP) + decoder.decompress(columnVector, input.length) if (input.nonEmpty) { - val numNulls = ByteBufferHelper.getInt(nullsBuffer) - var cntNulls = 0 - var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) input.zipWithIndex.foreach { case (expected: Any, index: Int) if expected == nullValue => - assertResult(index, "Wrong null position") { - nullPos - } - decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) - cntNulls += 1 - if (cntNulls < numNulls) { - nullPos = ByteBufferHelper.getInt(nullsBuffer) + assertResult(true, s"Wrong null ${index}th-position") { + columnVector.isNullAt(index) } case (expected: Int, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded int value") { - ByteBufferHelper.getInt(decodeBuffer) + columnVector.getInt(index) } case (expected: Long, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded long value") { - ByteBufferHelper.getLong(decodeBuffer) + columnVector.getLong(index) } case _ => fail("Unsupported type") } } - assert(!decodeBuffer.hasRemaining) } test(s"$scheme: empty column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index 437750fd2364d..0512e081544b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types.{AtomicType, IntegralType} class PassThroughSuite extends SparkFunSuite { @@ -116,50 +118,43 @@ class PassThroughSuite extends SparkFunSuite { assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = PassThrough.decoder(buffer, columnType) - val (decodeBuffer, nullsBuffer) = decoder.decompress(input.length) + val columnVector = ColumnVector.allocate(input.length, columnType.dataType, + MemoryMode.ON_HEAP) + decoder.decompress(columnVector, input.length) if (input.nonEmpty) { - val numNulls = ByteBufferHelper.getInt(nullsBuffer) - var cntNulls = 0 - var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) input.zipWithIndex.foreach { case (expected: Any, index: Int) if expected == nullValue => - assertResult(index, "Wrong null position") { - nullPos - } - decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) - cntNulls += 1 - if (cntNulls < numNulls) { - nullPos = ByteBufferHelper.getInt(nullsBuffer) + assertResult(true, s"Wrong null ${index}th-position") { + columnVector.isNullAt(index) } case (expected: Byte, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded byte value") { - decodeBuffer.get() + columnVector.getByte(index) } case (expected: Short, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded short value") { - ByteBufferHelper.getShort(decodeBuffer) + columnVector.getShort(index) } case (expected: Int, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded int value") { - ByteBufferHelper.getInt(decodeBuffer) + columnVector.getInt(index) } case (expected: Long, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded long value") { - ByteBufferHelper.getLong(decodeBuffer) + columnVector.getLong(index) } case (expected: Float, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded float value") { - ByteBufferHelper.getFloat(decodeBuffer) + columnVector.getFloat(index) } case (expected: Double, index: Int) => assertResult(expected, s"Wrong ${index}-th decoded double value") { - ByteBufferHelper.getDouble(decodeBuffer) + columnVector.getDouble(index) } case _ => fail("Unsupported type") } } - assert(!decodeBuffer.hasRemaining) } test(s"$PassThrough with $typeName: empty column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index dea240fb663f6..214551b8c2a96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types.AtomicType class RunLengthEncodingSuite extends SparkFunSuite { @@ -125,50 +127,43 @@ class RunLengthEncodingSuite extends SparkFunSuite { assertResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = RunLengthEncoding.decoder(buffer, columnType) - val (decodeBuffer, nullsBuffer) = decoder.decompress(inputSeq.length) + val columnVector = ColumnVector.allocate(inputSeq.length, columnType.dataType, + MemoryMode.ON_HEAP) + decoder.decompress(columnVector, inputSeq.length) if (inputSeq.nonEmpty) { - val numNulls = ByteBufferHelper.getInt(nullsBuffer) - var cntNulls = 0 - var nullPos = if (numNulls == 0) -1 else ByteBufferHelper.getInt(nullsBuffer) inputSeq.zipWithIndex.foreach { case (expected: Any, index: Int) if expected == nullValue => - assertResult(index, "Wrong null position") { - nullPos - } - decodeBuffer.position(decodeBuffer.position + columnType.defaultSize) - cntNulls += 1 - if (cntNulls < numNulls) { - nullPos = ByteBufferHelper.getInt(nullsBuffer) + assertResult(true, s"Wrong null ${index}th-position") { + columnVector.isNullAt(index) } case (i: Int, index: Int) => columnType match { case BOOLEAN => assertResult(values(i), s"Wrong ${index}-th decoded boolean value") { - if (decodeBuffer.get() == 1) true else false + columnVector.getBoolean(index) } case BYTE => assertResult(values(i), s"Wrong ${index}-th decoded byte value") { - decodeBuffer.get() + columnVector.getByte(index) } case SHORT => assertResult(values(i), s"Wrong ${index}-th decoded short value") { - ByteBufferHelper.getShort(decodeBuffer) + columnVector.getShort(index) } case INT => assertResult(values(i), s"Wrong ${index}-th decoded int value") { - ByteBufferHelper.getInt(decodeBuffer) + columnVector.getInt(index) } case LONG => assertResult(values(i), s"Wrong ${index}-th decoded long value") { - ByteBufferHelper.getLong(decodeBuffer) + columnVector.getLong(index) } case _ => fail("Unsupported type") } case _ => fail("Unsupported type") } } - assert(!decodeBuffer.hasRemaining) } test(s"$RunLengthEncoding with $typeName: empty column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 32c178debf455..e39fc1eb88614 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.execution.columnar.ColumnAccessor import org.apache.spark.sql.execution.columnar.compression.ColumnBuilderHelper import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform @@ -1308,8 +1309,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { @@ -1331,8 +1333,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { @@ -1354,8 +1357,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { @@ -1377,8 +1381,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { @@ -1400,8 +1405,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { @@ -1423,8 +1429,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { @@ -1446,8 +1453,9 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = new CachedBatchColumnVector( - JavaUtils.bufferToArray(columnBuilder.build), 16, dataType) + val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, column, 16) assert(column.isNullAt(0) == true) for (i <- 1 until 16) { From ff1ca2310c130d80346f42954043602eeec7e0e7 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 22 Aug 2017 07:28:02 +0100 Subject: [PATCH 07/17] fix scala type error --- .../spark/sql/execution/vectorized/ColumnarBatchSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index e39fc1eb88614..9783a11de8b15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -29,10 +29,10 @@ import org.apache.arrow.vector.NullableIntVector import org.apache.spark.SparkFunSuite import org.apache.spark.memory.MemoryMode -import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.execution.arrow.ArrowUtils import org.apache.spark.sql.execution.columnar.ColumnAccessor import org.apache.spark.sql.execution.columnar.compression.ColumnBuilderHelper import org.apache.spark.sql.types._ From 9c4e1e01746754b8a36ebb779c0e30188cbb0e2b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 22 Aug 2017 12:10:12 +0100 Subject: [PATCH 08/17] fix test failure of RunLengthEncoding --- .../sql/execution/columnar/compression/compressionSchemes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 59ed0e4f7e086..183e679fc8934 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -420,9 +420,9 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } columnVector.putNull(pos) } + pos += 1 } case _: LongType => - val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) var currentValueLocal: Long = 0 while (valueCountLocal < runLocal || (pos < capacity)) { if (pos != nextNullIndex) { From 55abc6f7e3ce7b796bda6790f8aeb3e1f076713c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 24 Aug 2017 16:56:39 +0100 Subject: [PATCH 09/17] rebase with master --- .../vectorized/WritableColumnVector.java | 15 +++++++++++++++ .../sql/execution/columnar/ColumnAccessor.scala | 5 +++-- .../compression/CompressibleColumnAccessor.scala | 4 ++-- .../columnar/compression/CompressionScheme.scala | 4 ++-- .../columnar/compression/compressionSchemes.scala | 14 +++++++------- .../columnar/compression/BooleanBitSetSuite.scala | 5 ++--- .../compression/DictionaryEncodingSuite.scala | 6 ++---- .../columnar/compression/IntegralDeltaSuite.scala | 6 ++---- .../compression/PassThroughEncodingSuite.scala | 8 +++----- .../compression/RunLengthEncodingSuite.scala | 6 ++---- .../execution/vectorized/ColumnarBatchSuite.scala | 14 +++++++------- 11 files changed, 47 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index 163f2511e5f73..b138b9850ba6d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -164,6 +164,11 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { */ public abstract void putShorts(int rowId, int count, short[] src, int srcIndex); + /** + * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + */ + public abstract void putShorts(int rowId, int count, byte[] src, int srcIndex); + /** * Sets the value at rowId to `value`. */ @@ -179,6 +184,11 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { */ public abstract void putInts(int rowId, int count, int[] src, int srcIndex); + /** + * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + */ + public abstract void putInts(int rowId, int count, byte[] src, int srcIndex); + /** * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) * The data in src must be 4-byte little endian ints. @@ -200,6 +210,11 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { */ public abstract void putLongs(int rowId, int count, long[] src, int srcIndex); + /** + * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + */ + public abstract void putLongs(int rowId, int count, byte[] src, int srcIndex); + /** * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) * The data in src must be 8-byte little endian longs. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index 53e7687a3fb96..c90a88762c252 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -24,7 +24,7 @@ import scala.annotation.tailrec import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeMapData, UnsafeRow} import org.apache.spark.sql.execution.columnar.compression.CompressibleColumnAccessor -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types._ /** @@ -154,7 +154,8 @@ private[sql] object ColumnAccessor { } } - def decompress(columnAccessor: ColumnAccessor, columnVector: ColumnVector, numRows: Int): Unit = { + def decompress(columnAccessor: ColumnAccessor, columnVector: WritableColumnVector, numRows: Int): + Unit = { if (columnAccessor.isInstanceOf[NativeColumnAccessor[_]]) { val nativeAccessor = columnAccessor.asInstanceOf[NativeColumnAccessor[_]] nativeAccessor.decompress(columnVector, numRows) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala index 856a4b5dc099a..c1ff83dca6823 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnAccessor, NativeColumnAccessor} -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types.AtomicType private[columnar] trait CompressibleColumnAccessor[T <: AtomicType] extends ColumnAccessor { @@ -40,6 +40,6 @@ private[columnar] trait CompressibleColumnAccessor[T <: AtomicType] extends Colu decoder.next(row, ordinal) } - def decompress(columnVector: ColumnVector, capacity: Int): Unit = + def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = decoder.decompress(columnVector, capacity) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala index c471694a1f379..f8aeba44257d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressionScheme.scala @@ -21,7 +21,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnType, NativeColumnType} -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types.AtomicType private[columnar] trait Encoder[T <: AtomicType] { @@ -43,7 +43,7 @@ private[columnar] trait Decoder[T <: AtomicType] { def hasNext: Boolean - def decompress(columnVector: ColumnVector, capacity: Int): Unit + def decompress(columnVector: WritableColumnVector, capacity: Int): Unit } private[columnar] trait CompressionScheme { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 183e679fc8934..57e37ee90bf29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -25,7 +25,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.columnar._ -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types._ @@ -64,7 +64,7 @@ private[columnar] case object PassThrough extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -328,7 +328,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { override def hasNext: Boolean = valueCount < run || buffer.hasRemaining - override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -556,7 +556,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -686,7 +686,7 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { override def hasNext: Boolean = visited < count - override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { val countLocal = count var currentWordLocal: Long = 0 var visitedLocal: Int = 0 @@ -798,7 +798,7 @@ private[columnar] case object IntDelta extends CompressionScheme { row.setInt(ordinal, prev) } - override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { var prevLocal: Int = 0 val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() @@ -904,7 +904,7 @@ private[columnar] case object LongDelta extends CompressionScheme { row.setLong(ordinal, prev) } - override def decompress(columnVector: ColumnVector, capacity: Int): Unit = { + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { var prevLocal: Long = 0 val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala index 52ab1aa57d900..2d71a42628dfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar.{BOOLEAN, NoopColumnStats} import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.types.BooleanType class BooleanBitSetSuite extends SparkFunSuite { @@ -106,7 +105,7 @@ class BooleanBitSetSuite extends SparkFunSuite { assertResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) - val columnVector = ColumnVector.allocate(values.length, BooleanType, MemoryMode.ON_HEAP) + val columnVector = new OnHeapColumnVector(values.length, BooleanType) decoder.decompress(columnVector, values.length) if (values.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index e99f1c235d340..28950b74cf1c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -20,11 +20,10 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.ByteBuffer import org.apache.spark.SparkFunSuite -import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.types.AtomicType class DictionaryEncodingSuite extends SparkFunSuite { @@ -143,8 +142,7 @@ class DictionaryEncodingSuite extends SparkFunSuite { assertResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = DictionaryEncoding.decoder(buffer, columnType) - val columnVector = ColumnVector.allocate(inputSeq.length, columnType.dataType, - MemoryMode.ON_HEAP) + val columnVector = new OnHeapColumnVector(inputSeq.length, columnType.dataType) decoder.decompress(columnVector, inputSeq.length) if (inputSeq.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index c3d1eba019cd4..c39c227384755 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.types.IntegralType class IntegralDeltaSuite extends SparkFunSuite { @@ -136,8 +135,7 @@ class IntegralDeltaSuite extends SparkFunSuite { assertResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = scheme.decoder(buffer, columnType) - val columnVector = ColumnVector.allocate(input.length, columnType.dataType, - MemoryMode.ON_HEAP) + val columnVector = new OnHeapColumnVector(input.length, columnType.dataType) decoder.decompress(columnVector, input.length) if (input.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index 0512e081544b5..ab39b9c630128 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.execution.vectorized.ColumnVector -import org.apache.spark.sql.types.{AtomicType, IntegralType} +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.types.AtomicType class PassThroughSuite extends SparkFunSuite { val nullValue = -1 @@ -118,8 +117,7 @@ class PassThroughSuite extends SparkFunSuite { assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = PassThrough.decoder(buffer, columnType) - val columnVector = ColumnVector.allocate(input.length, columnType.dataType, - MemoryMode.ON_HEAP) + val columnVector = new OnHeapColumnVector(input.length, columnType.dataType) decoder.decompress(columnVector, input.length) if (input.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index 214551b8c2a96..2263bb43b82b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.types.AtomicType class RunLengthEncodingSuite extends SparkFunSuite { @@ -127,8 +126,7 @@ class RunLengthEncodingSuite extends SparkFunSuite { assertResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val decoder = RunLengthEncoding.decoder(buffer, columnType) - val columnVector = ColumnVector.allocate(inputSeq.length, columnType.dataType, - MemoryMode.ON_HEAP) + val columnVector = new OnHeapColumnVector(inputSeq.length, columnType.dataType) decoder.decompress(columnVector, inputSeq.length) if (inputSeq.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 9783a11de8b15..bc74664567943 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -1309,7 +1309,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) @@ -1333,7 +1333,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) @@ -1357,7 +1357,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) @@ -1381,7 +1381,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) @@ -1405,7 +1405,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) @@ -1429,7 +1429,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) @@ -1453,7 +1453,7 @@ class ColumnarBatchSuite extends SparkFunSuite { columnBuilder.appendFrom(row, 0) } - val column = ColumnVector.allocate(16, dataType, MemoryMode.ON_HEAP) + val column = allocate(16, dataType, MemoryMode.ON_HEAP) val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, column, 16) From 133375d26d431b7fba64be20995ebbcbbcff77ce Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 24 Aug 2017 17:32:18 +0100 Subject: [PATCH 10/17] Delay decompress for DictionaryEncoding --- .../execution/columnar/ColumnDictionary.java | 53 +++++++++++++++++++ .../compression/compressionSchemes.scala | 14 ++--- 2 files changed, 60 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java b/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java new file mode 100644 index 0000000000000..a2065dfe9dba7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java @@ -0,0 +1,53 @@ +/* + * 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.execution.columnar; + +import org.apache.spark.sql.execution.vectorized.Dictionary; + +public final class ColumnDictionary implements Dictionary { + private Object[] dictionary; + + public ColumnDictionary(Object[] dictionary) { + this.dictionary = dictionary; + } + + @Override + public int decodeToInt(int id) { + return (Integer)dictionary[id]; + } + + @Override + public long decodeToLong(int id) { + return (Long)dictionary[id]; + } + + @Override + public float decodeToFloat(int id) { + throw new UnsupportedOperationException("Dictionary encoding does not support float"); + } + + @Override + public double decodeToDouble(int id) { + throw new UnsupportedOperationException("Dictionary encoding does not support double"); + } + + @Override + public byte[] decodeToBinary(int id) { + throw new UnsupportedOperationException("Dictionary encoding does not support String"); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 57e37ee90bf29..3765c89ec46fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -565,24 +565,24 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { var seenNulls = 0 columnType.dataType match { case _: IntegerType => + val dictionaryIds = columnVector.reserveDictionaryIds(capacity) + columnVector.setDictionary(new ColumnDictionary(dictionary.asInstanceOf[Array[AnyRef]])) while (pos < capacity) { if (pos != nextNullIndex) { - val value = dictionary(buffer.getShort()).asInstanceOf[Int] - columnVector.putInt(pos, value) + dictionaryIds.putInt(pos, buffer.getShort()) } else { seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } + if (seenNulls < nullCount) nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) columnVector.putNull(pos) } pos += 1 } case _: LongType => + val dictionaryIds = columnVector.reserveDictionaryIds(capacity) + columnVector.setDictionary(new ColumnDictionary(dictionary.asInstanceOf[Array[AnyRef]])) while (pos < capacity) { if (pos != nextNullIndex) { - val value = dictionary(buffer.getShort()).asInstanceOf[Long] - columnVector.putLong(pos, value) + dictionaryIds.putInt(pos, buffer.getShort()) } else { seenNulls += 1 if (seenNulls < nullCount) { From 4ef81e553058cec9ebd8b4e624913842daa92ff5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Sep 2017 10:06:50 +0100 Subject: [PATCH 11/17] address review comments --- .../execution/columnar/ColumnDictionary.java | 15 +- .../execution/columnar/ColumnAccessor.scala | 9 - .../compression/compressionSchemes.scala | 235 ++++++++---------- 3 files changed, 108 insertions(+), 151 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java b/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java index a2065dfe9dba7..f1785853a94ae 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java @@ -20,20 +20,25 @@ import org.apache.spark.sql.execution.vectorized.Dictionary; public final class ColumnDictionary implements Dictionary { - private Object[] dictionary; + private int[] intDictionary; + private long[] longDictionary; - public ColumnDictionary(Object[] dictionary) { - this.dictionary = dictionary; + public ColumnDictionary(int[] dictionary) { + this.intDictionary = dictionary; + } + + public ColumnDictionary(long[] dictionary) { + this.longDictionary = dictionary; } @Override public int decodeToInt(int id) { - return (Integer)dictionary[id]; + return intDictionary[id]; } @Override public long decodeToLong(int id) { - return (Long)dictionary[id]; + return longDictionary[id]; } @Override diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index c90a88762c252..24c8ac81420cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -160,15 +160,6 @@ private[sql] object ColumnAccessor { val nativeAccessor = columnAccessor.asInstanceOf[NativeColumnAccessor[_]] nativeAccessor.decompress(columnVector, numRows) } else { - val dataBuffer = columnAccessor.asInstanceOf[BasicColumnAccessor[_]].getByteBuffer - val nullsBuffer = dataBuffer.duplicate().order(ByteOrder.nativeOrder()) - nullsBuffer.rewind() - - val numNulls = ByteBufferHelper.getInt(nullsBuffer) - for (i <- 0 until numNulls) { - val cordinal = ByteBufferHelper.getInt(nullsBuffer) - columnVector.putNull(cordinal) - } throw new RuntimeException("Not support non-primitive type now") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 3765c89ec46fe..285d7eef35abf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -64,7 +64,48 @@ private[columnar] case object PassThrough extends CompressionScheme { override def hasNext: Boolean = buffer.hasRemaining - override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { + private def putBooleans( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + for (i <- 0 until len) { + columnVector.putBoolean(pos + i, (buffer.get(bufferPos + i) != 0)) + } + } + + private def putBytes( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + columnVector.putBytes(pos, len, buffer.array, bufferPos) + } + + private def putShorts( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + columnVector.putShorts(pos, len, buffer.array, bufferPos) + } + + private def putInts( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + columnVector.putInts(pos, len, buffer.array, bufferPos) + } + + private def putLongs( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + columnVector.putLongs(pos, len, buffer.array, bufferPos) + } + + private def putFloats( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + columnVector.putFloats(pos, len, buffer.array, bufferPos) + } + + private def putDoubles( + columnVector: WritableColumnVector, pos: Int, bufferPos: Int, len: Int): Unit = { + columnVector.putDoubles(pos, len, buffer.array, bufferPos) + } + + private def decompress0( + columnVector: WritableColumnVector, + capacity: Int, + unitSize: Int, + putFunction: (WritableColumnVector, Int, Int, Int) => Unit): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -73,150 +114,49 @@ private[columnar] case object PassThrough extends CompressionScheme { var seenNulls = 0 val srcArray = buffer.array var bufferPos = buffer.position + while (pos < capacity) { + if (pos != nextNullIndex) { + val len = nextNullIndex - pos + assert(len * unitSize < Int.MaxValue) + putFunction(columnVector, pos, bufferPos, len) + bufferPos += len * unitSize + pos += len + } else { + seenNulls += 1 + nextNullIndex = if (seenNulls < nullCount) { + ByteBufferHelper.getInt(nullsBuffer) + } else { + capacity + } + columnVector.putNull(pos) + pos += 1 + } + } + } + + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { columnType.dataType match { case _: BooleanType => val unitSize = 1 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - for (i <- 0 until len) { - val value = buffer.get(bufferPos + i) != 0 - columnVector.putBoolean(pos + i, value) - } - bufferPos += len - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putBooleans) case _: ByteType => val unitSize = 1 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - columnVector.putBytes(pos, len, srcArray, bufferPos) - bufferPos += len * unitSize - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putBytes) case _: ShortType => val unitSize = 2 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - columnVector.putShorts(pos, len, srcArray, bufferPos) - bufferPos += len * unitSize - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putShorts) case _: IntegerType => val unitSize = 4 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - columnVector.putInts(pos, len, srcArray, bufferPos) - bufferPos += len * unitSize - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putInts) case _: LongType => val unitSize = 8 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - columnVector.putLongs(pos, len, srcArray, bufferPos) - bufferPos += len * unitSize - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putLongs) case _: FloatType => val unitSize = 4 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - columnVector.putFloats(pos, len, srcArray, bufferPos) - bufferPos += len * unitSize - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putFloats) case _: DoubleType => val unitSize = 8 - while (pos < capacity) { - if (pos != nextNullIndex) { - val len = nextNullIndex - pos - assert(len * unitSize < Int.MaxValue) - columnVector.putDoubles(pos, len, srcArray, bufferPos) - bufferPos += len * unitSize - pos += len - } else { - seenNulls += 1 - nextNullIndex = if (seenNulls < nullCount) { - ByteBufferHelper.getInt(nullsBuffer) - } else { - capacity - } - columnVector.putNull(pos) - pos += 1 - } - } + decompress0(columnVector, capacity, unitSize, putDoubles) } } } @@ -543,11 +483,32 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { } class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) - extends compression.Decoder[T] { - - private val dictionary: Array[Any] = { - val elementNum = ByteBufferHelper.getInt(buffer) - Array.fill[Any](elementNum)(columnType.extract(buffer).asInstanceOf[Any]) + extends compression.Decoder[T] { + val elementNum = ByteBufferHelper.getInt(buffer) + private val dictionary: Array[Any] = new Array[Any](elementNum) + private var intDictionary: Array[Int] = null + private var longDictionary: Array[Long] = null + + columnType.dataType match { + case _: IntegerType => + intDictionary = new Array[Int](elementNum) + for (i <- 0 until elementNum) { + val v = columnType.extract(buffer).asInstanceOf[Int] + intDictionary(i) = v + dictionary(i) = v + } + case _: LongType => + longDictionary = new Array[Long](elementNum) + for (i <- 0 until elementNum) { + val v = columnType.extract(buffer).asInstanceOf[Long] + longDictionary(i) = v + dictionary(i) = v + } + case _: StringType => + for (i <- 0 until elementNum) { + val v = columnType.extract(buffer).asInstanceOf[Any] + dictionary(i) = v + } } override def next(row: InternalRow, ordinal: Int): Unit = { @@ -566,7 +527,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { columnType.dataType match { case _: IntegerType => val dictionaryIds = columnVector.reserveDictionaryIds(capacity) - columnVector.setDictionary(new ColumnDictionary(dictionary.asInstanceOf[Array[AnyRef]])) + columnVector.setDictionary(new ColumnDictionary(intDictionary)) while (pos < capacity) { if (pos != nextNullIndex) { dictionaryIds.putInt(pos, buffer.getShort()) @@ -579,7 +540,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { } case _: LongType => val dictionaryIds = columnVector.reserveDictionaryIds(capacity) - columnVector.setDictionary(new ColumnDictionary(dictionary.asInstanceOf[Array[AnyRef]])) + columnVector.setDictionary(new ColumnDictionary(longDictionary)) while (pos < capacity) { if (pos != nextNullIndex) { dictionaryIds.putInt(pos, buffer.getShort()) From 9fa1b2874471a55413d9d1fbbcd3021d06198961 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 18 Sep 2017 16:20:03 +0100 Subject: [PATCH 12/17] remove unused import revert unexpected style change --- .../columnar/compression/CompressibleColumnAccessor.scala | 2 -- .../execution/columnar/compression/IntegralDeltaSuite.scala | 6 +++--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala index c1ff83dca6823..774011f1e3de8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnAccessor.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.columnar.compression -import java.nio.ByteBuffer - import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnAccessor, NativeColumnAccessor} import org.apache.spark.sql.execution.vectorized.WritableColumnVector diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index c39c227384755..0d9f1fb0c02c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -30,9 +30,9 @@ class IntegralDeltaSuite extends SparkFunSuite { testIntegralDelta(new LongColumnStats, LONG, LongDelta) def testIntegralDelta[I <: IntegralType]( - columnStats: ColumnStats, - columnType: NativeColumnType[I], - scheme: CompressionScheme) { + columnStats: ColumnStats, + columnType: NativeColumnType[I], + scheme: CompressionScheme) { def skeleton(input: Seq[I#InternalType]) { // ------------- From 4cb6823582b9ef7b3177f433e69667f96c6fd076 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 18 Sep 2017 16:20:27 +0100 Subject: [PATCH 13/17] reduce code duplication --- .../compression/compressionSchemes.scala | 172 +++++++----------- 1 file changed, 70 insertions(+), 102 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 285d7eef35abf..bf00ad997c76e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -112,7 +112,6 @@ private[columnar] case object PassThrough extends CompressionScheme { var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else capacity var pos = 0 var seenNulls = 0 - val srcArray = buffer.array var bufferPos = buffer.position while (pos < capacity) { if (pos != nextNullIndex) { @@ -268,7 +267,47 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { override def hasNext: Boolean = valueCount < run || buffer.hasRemaining - override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { + private def putBoolean(columnVector: WritableColumnVector, pos: Int, value: Long): Unit = { + columnVector.putBoolean(pos, value == 1) + } + + private def getByte(buffer: ByteBuffer): Long = { + buffer.get().toLong + } + + private def putByte(columnVector: WritableColumnVector, pos: Int, value: Long): Unit = { + columnVector.putByte(pos, value.toByte) + } + + private def getShort(buffer: ByteBuffer): Long = { + buffer.getShort().toLong + } + + private def putShort(columnVector: WritableColumnVector, pos: Int, value: Long): Unit = { + columnVector.putShort(pos, value.toShort) + } + + private def getInt(buffer: ByteBuffer): Long = { + buffer.getInt().toLong + } + + private def putInt(columnVector: WritableColumnVector, pos: Int, value: Long): Unit = { + columnVector.putInt(pos, value.toInt) + } + + private def getLong(buffer: ByteBuffer): Long = { + buffer.getLong() + } + + private def putLong(columnVector: WritableColumnVector, pos: Int, value: Long): Unit = { + columnVector.putLong(pos, value) + } + + private def decompress0( + columnVector: WritableColumnVector, + capacity: Int, + getFunction: (ByteBuffer) => Long, + putFunction: (WritableColumnVector, Int, Long) => Unit): Unit = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) @@ -277,112 +316,41 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { var seenNulls = 0 var runLocal = 0 var valueCountLocal = 0 + var currentValueLocal: Long = 0 + + while (valueCountLocal < runLocal || (pos < capacity)) { + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = getFunction(buffer) + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + putFunction(columnVector, pos, currentValueLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + columnVector.putNull(pos) + } + pos += 1 + } + } + + override def decompress(columnVector: WritableColumnVector, capacity: Int): Unit = { columnType.dataType match { case _: BooleanType => - var currentValueLocal: Boolean = false - while (valueCountLocal < runLocal || (pos < capacity)) { - if (pos != nextNullIndex) { - if (valueCountLocal == runLocal) { - currentValueLocal = buffer.get() == 1 - runLocal = ByteBufferHelper.getInt(buffer) - valueCountLocal = 1 - } else { - valueCountLocal += 1 - } - columnVector.putBoolean(pos, currentValueLocal) - } else { - seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } - columnVector.putNull(pos) - } - pos += 1 - } + decompress0(columnVector, capacity, getByte, putBoolean) case _: ByteType => - var currentValueLocal: Byte = 0 - while (valueCountLocal < runLocal || (pos < capacity)) { - if (pos != nextNullIndex) { - if (valueCountLocal == runLocal) { - currentValueLocal = buffer.get() - runLocal = ByteBufferHelper.getInt(buffer) - valueCountLocal = 1 - } else { - valueCountLocal += 1 - } - columnVector.putByte(pos, currentValueLocal) - } else { - seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } - columnVector.putNull(pos) - } - pos += 1 - } + decompress0(columnVector, capacity, getByte, putByte) case _: ShortType => - var currentValueLocal: Short = 0 - while (valueCountLocal < runLocal || (pos < capacity)) { - if (pos != nextNullIndex) { - if (valueCountLocal == runLocal) { - currentValueLocal = buffer.getShort() - runLocal = ByteBufferHelper.getInt(buffer) - valueCountLocal = 1 - } else { - valueCountLocal += 1 - } - columnVector.putShort(pos, currentValueLocal) - } else { - seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } - columnVector.putNull(pos) - } - pos += 1 - } + decompress0(columnVector, capacity, getShort, putShort) case _: IntegerType => - var currentValueLocal: Int = 0 - while (valueCountLocal < runLocal || (pos < capacity)) { - if (pos != nextNullIndex) { - if (valueCountLocal == runLocal) { - currentValueLocal = buffer.getInt() - runLocal = ByteBufferHelper.getInt(buffer) - valueCountLocal = 1 - } else { - valueCountLocal += 1 - } - columnVector.putInt(pos, currentValueLocal) - } else { - seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } - columnVector.putNull(pos) - } - pos += 1 - } + decompress0(columnVector, capacity, getInt, putInt) case _: LongType => - var currentValueLocal: Long = 0 - while (valueCountLocal < runLocal || (pos < capacity)) { - if (pos != nextNullIndex) { - if (valueCountLocal == runLocal) { - currentValueLocal = buffer.getLong() - runLocal = ByteBufferHelper.getInt(buffer) - valueCountLocal = 1 - } else { - valueCountLocal += 1 - } - columnVector.putLong(pos, currentValueLocal) - } else { - seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } - columnVector.putNull(pos) - } - pos += 1 - } + decompress0(columnVector, capacity, getLong, putLong) case _ => throw new IllegalStateException("Not supported type in RunLengthEncoding.") } } From fc12bd83412b2926febcdc3b35b46bcf08d6250b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 18 Sep 2017 16:21:38 +0100 Subject: [PATCH 14/17] update comments --- .../vectorized/WritableColumnVector.java | 67 ++++++++++--------- 1 file changed, 35 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index b138b9850ba6d..da72954ddc448 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -113,153 +113,156 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { protected abstract void reserveInternal(int capacity); /** - * Sets the value at rowId to null/not null. + * Sets null/not null to the value at rowId. */ public abstract void putNotNull(int rowId); public abstract void putNull(int rowId); /** - * Sets the values from [rowId, rowId + count) to null/not null. + * Sets null/not null to the values at [rowId, rowId + count). */ public abstract void putNulls(int rowId, int count); public abstract void putNotNulls(int rowId, int count); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putBoolean(int rowId, boolean value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putBooleans(int rowId, int count, boolean value); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putByte(int rowId, byte value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putBytes(int rowId, int count, byte value); /** - * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count) */ public abstract void putBytes(int rowId, int count, byte[] src, int srcIndex); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putShort(int rowId, short value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putShorts(int rowId, int count, short value); /** - * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count) */ public abstract void putShorts(int rowId, int count, short[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + * Sets values from [src[srcIndex], src[srcIndex + count * 2]) to [rowId, rowId + count) + * The data in src must be 2-byte platform native endian shorts. */ public abstract void putShorts(int rowId, int count, byte[] src, int srcIndex); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putInt(int rowId, int value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putInts(int rowId, int count, int value); /** - * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count) */ public abstract void putInts(int rowId, int count, int[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + * Sets values from [src[srcIndex], src[srcIndex + count * 4]) to [rowId, rowId + count) + * The data in src must be 4-byte platform native endian ints. */ public abstract void putInts(int rowId, int count, byte[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + * Sets values from [src[srcIndex], src[srcIndex + count * 4]) to [rowId, rowId + count) * The data in src must be 4-byte little endian ints. */ public abstract void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putLong(int rowId, long value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putLongs(int rowId, int count, long value); /** - * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count) */ public abstract void putLongs(int rowId, int count, long[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + * Sets values from [src[srcIndex], src[srcIndex + count * 8]) to [rowId, rowId + count) + * The data in src must be 8-byte platform native endian longs. */ public abstract void putLongs(int rowId, int count, byte[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) + * Sets values from [src + srcIndex, src + srcIndex + count * 8) to [rowId, rowId + count) * The data in src must be 8-byte little endian longs. */ public abstract void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putFloat(int rowId, float value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putFloats(int rowId, int count, float value); /** - * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count) */ public abstract void putFloats(int rowId, int count, float[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) - * The data in src must be ieee formatted floats. + * Sets values from [src[srcIndex], src[srcIndex + count * 4]) to [rowId, rowId + count) + * The data in src must be ieee formatted floats in platform native endian. */ public abstract void putFloats(int rowId, int count, byte[] src, int srcIndex); /** - * Sets the value at rowId to `value`. + * Sets `value` to the value at rowId. */ public abstract void putDouble(int rowId, double value); /** - * Sets values from [rowId, rowId + count) to value. + * Sets value to [rowId, rowId + count). */ public abstract void putDoubles(int rowId, int count, double value); /** - * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count) */ public abstract void putDoubles(int rowId, int count, double[] src, int srcIndex); /** - * Sets values from [rowId, rowId + count) to [src[srcIndex], src[srcIndex + count]) - * The data in src must be ieee formatted doubles. + * Sets values from [src[srcIndex], src[srcIndex + count * 8]) to [rowId, rowId + count) + * The data in src must be ieee formatted doubles in platform native endian. */ public abstract void putDoubles(int rowId, int count, byte[] src, int srcIndex); @@ -269,7 +272,7 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { public abstract void putArray(int rowId, int offset, int length); /** - * Sets the value at rowId to `value`. + * Sets values from [value + offset, value + offset + count) to the values at rowId. */ public abstract int putByteArray(int rowId, byte[] value, int offset, int count); public final int putByteArray(int rowId, byte[] value) { From 9dd58d3a498694091ef07a9bf24330ecf89889b5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Sep 2017 12:14:47 +0100 Subject: [PATCH 15/17] address review comment --- .../vectorized/ColumnVectorSuite.scala | 182 +++++++++++++++++- .../vectorized/ColumnarBatchSuite.scala | 173 +---------------- 2 files changed, 181 insertions(+), 174 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 85da8270d4cba..1e290c2b89d47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -20,7 +20,10 @@ package org.apache.spark.sql.execution.vectorized import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.execution.columnar.ColumnAccessor +import org.apache.spark.sql.execution.columnar.compression.ColumnBuilderHelper import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -31,14 +34,21 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { try block(vector) finally vector.close() } + private def withVectors( + size: Int, + dt: DataType)( + block: WritableColumnVector => Unit): Unit = { + withVector(new OnHeapColumnVector(size, dt))(block) + withVector(new OffHeapColumnVector(size, dt))(block) + } + private def testVectors( name: String, size: Int, dt: DataType)( block: WritableColumnVector => Unit): Unit = { test(name) { - withVector(new OnHeapColumnVector(size, dt))(block) - withVector(new OffHeapColumnVector(size, dt))(block) + withVectors(size, dt)(block) } } @@ -218,4 +228,172 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { (0 until 8).foreach(i => assert(testVector.isNullAt(i) == (i % 2 == 0))) } } + + test("CachedBatch boolean Apis") { + val dataType = BooleanType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setBoolean(0, i % 2 == 0) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getBoolean(i) == (i % 2 == 0)) + } + } + } + + test("CachedBatch byte Apis") { + val dataType = ByteType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setByte(0, i.toByte) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getByte(i) == i) + } + } + } + + test("CachedBatch short Apis") { + val dataType = ShortType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setShort(0, i.toShort) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getShort(i) == i) + } + } + } + + test("CachedBatch int Apis") { + val dataType = IntegerType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setInt(0, i) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getInt(i) == i) + } + } + } + + test("CachedBatch long Apis") { + val dataType = LongType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setLong(0, i.toLong) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getLong(i) == i.toLong) + } + } + } + + test("CachedBatch float Apis") { + val dataType = FloatType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setFloat(0, i.toFloat) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getFloat(i) == i.toFloat) + } + } + } + + test("CachedBatch double Apis") { + val dataType = DoubleType + val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) + val row = new SpecificInternalRow(Array(dataType)) + + row.setNullAt(0) + columnBuilder.appendFrom(row, 0) + for (i <- 1 until 16) { + row.setDouble(0, i.toDouble) + columnBuilder.appendFrom(row, 0) + } + + withVectors(16, dataType) { testVector => + val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) + ColumnAccessor.decompress(columnAccessor, testVector, 16) + + assert(testVector.isNullAt(0) == true) + for (i <- 1 until 16) { + assert(testVector.isNullAt(i) == false) + assert(testVector.getDouble(i) == i.toDouble) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index bc74664567943..0b179aa97c479 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -31,13 +31,10 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.arrow.ArrowUtils -import org.apache.spark.sql.execution.columnar.ColumnAccessor -import org.apache.spark.sql.execution.columnar.compression.ColumnBuilderHelper import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.CalendarInterval class ColumnarBatchSuite extends SparkFunSuite { @@ -1296,172 +1293,4 @@ class ColumnarBatchSuite extends SparkFunSuite { batch.close() allocator.close() } - - test("CachedBatch boolean Apis") { - val dataType = BooleanType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setBoolean(0, i % 2 == 0) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getBoolean(i) == (i % 2 == 0)) - } - column.close - } - - test("CachedBatch byte Apis") { - val dataType = ByteType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setByte(0, i.toByte) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getByte(i) == i) - } - column.close - } - - test("CachedBatch short Apis") { - val dataType = ShortType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setShort(0, i.toShort) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getShort(i) == i) - } - column.close - } - - test("CachedBatch int Apis") { - val dataType = IntegerType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setInt(0, i) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getInt(i) == i) - } - column.close - } - - test("CachedBatch long Apis") { - val dataType = LongType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setLong(0, i.toLong) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getLong(i) == i.toLong) - } - column.close - } - - test("CachedBatch float Apis") { - val dataType = FloatType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setFloat(0, i.toFloat) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getFloat(i) == i.toFloat) - } - column.close - } - - test("CachedBatch double Apis") { - val dataType = DoubleType - val columnBuilder = ColumnBuilderHelper(dataType, 1024, "col", true) - val row = new SpecificInternalRow(Array(dataType)) - - row.setNullAt(0) - columnBuilder.appendFrom(row, 0) - for (i <- 1 until 16) { - row.setDouble(0, i.toDouble) - columnBuilder.appendFrom(row, 0) - } - - val column = allocate(16, dataType, MemoryMode.ON_HEAP) - val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) - ColumnAccessor.decompress(columnAccessor, column, 16) - - assert(column.isNullAt(0) == true) - for (i <- 1 until 16) { - assert(column.isNullAt(i) == false) - assert(column.getDouble(i) == i.toDouble) - } - column.close - } } From e9606df532dbbdc1a87be1415986157d7c50d805 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 24 Sep 2017 02:02:45 +0100 Subject: [PATCH 16/17] fix scala style error --- .../spark/sql/execution/vectorized/ColumnVectorSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 1e290c2b89d47..c5c8ae3a17c6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -397,3 +397,4 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } } } + From c16230d34472e0337b87ce858289fec9a1d88ab4 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 3 Oct 2017 18:37:46 +0100 Subject: [PATCH 17/17] address review comment --- .../PassThroughEncodingSuite.scala | 378 +++++++++--------- .../compression/RunLengthEncodingSuite.scala | 6 +- 2 files changed, 192 insertions(+), 192 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index ab39b9c630128..b6f0b5e6277b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -1,189 +1,189 @@ -/* - * 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.execution.columnar.compression - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.execution.columnar._ -import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector -import org.apache.spark.sql.types.AtomicType - -class PassThroughSuite extends SparkFunSuite { - val nullValue = -1 - testPassThrough(new ByteColumnStats, BYTE) - testPassThrough(new ShortColumnStats, SHORT) - testPassThrough(new IntColumnStats, INT) - testPassThrough(new LongColumnStats, LONG) - testPassThrough(new FloatColumnStats, FLOAT) - testPassThrough(new DoubleColumnStats, DOUBLE) - - def testPassThrough[T <: AtomicType]( - columnStats: ColumnStats, - columnType: NativeColumnType[T]) { - - val typeName = columnType.getClass.getSimpleName.stripSuffix("$") - - def skeleton(input: Seq[T#InternalType]) { - // ------------- - // Tests encoder - // ------------- - - val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) - - input.map { value => - val row = new GenericInternalRow(1) - columnType.setField(row, 0, value) - builder.appendFrom(row, 0) - } - - val buffer = builder.build() - // Column type ID + null count + null positions - val headerSize = CompressionScheme.columnHeaderSize(buffer) - - // Compression scheme ID + compressed contents - val compressedSize = 4 + input.size * columnType.defaultSize - - // 4 extra bytes for compression scheme type ID - assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) - - buffer.position(headerSize) - assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) - - if (input.nonEmpty) { - input.foreach { value => - assertResult(value, "Wrong value")(columnType.extract(buffer)) - } - } - - // ------------- - // Tests decoder - // ------------- - - // Rewinds, skips column header and 4 more bytes for compression scheme ID - buffer.rewind().position(headerSize + 4) - - val decoder = PassThrough.decoder(buffer, columnType) - val mutableRow = new GenericInternalRow(1) - - if (input.nonEmpty) { - input.foreach{ - assert(decoder.hasNext) - assertResult(_, "Wrong decoded value") { - decoder.next(mutableRow, 0) - columnType.getField(mutableRow, 0) - } - } - } - assert(!decoder.hasNext) - } - - def skeletonForDecompress(input: Seq[T#InternalType]) { - val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) - val row = new GenericInternalRow(1) - val nullRow = new GenericInternalRow(1) - nullRow.setNullAt(0) - input.map { value => - if (value == nullValue) { - builder.appendFrom(nullRow, 0) - } else { - columnType.setField(row, 0, value) - builder.appendFrom(row, 0) - } - } - val buffer = builder.build() - - // ---------------- - // Tests decompress - // ---------------- - // Rewinds, skips column header and 4 more bytes for compression scheme ID - val headerSize = CompressionScheme.columnHeaderSize(buffer) - buffer.position(headerSize) - assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) - - val decoder = PassThrough.decoder(buffer, columnType) - val columnVector = new OnHeapColumnVector(input.length, columnType.dataType) - decoder.decompress(columnVector, input.length) - - if (input.nonEmpty) { - input.zipWithIndex.foreach { - case (expected: Any, index: Int) if expected == nullValue => - assertResult(true, s"Wrong null ${index}th-position") { - columnVector.isNullAt(index) - } - case (expected: Byte, index: Int) => - assertResult(expected, s"Wrong ${index}-th decoded byte value") { - columnVector.getByte(index) - } - case (expected: Short, index: Int) => - assertResult(expected, s"Wrong ${index}-th decoded short value") { - columnVector.getShort(index) - } - case (expected: Int, index: Int) => - assertResult(expected, s"Wrong ${index}-th decoded int value") { - columnVector.getInt(index) - } - case (expected: Long, index: Int) => - assertResult(expected, s"Wrong ${index}-th decoded long value") { - columnVector.getLong(index) - } - case (expected: Float, index: Int) => - assertResult(expected, s"Wrong ${index}-th decoded float value") { - columnVector.getFloat(index) - } - case (expected: Double, index: Int) => - assertResult(expected, s"Wrong ${index}-th decoded double value") { - columnVector.getDouble(index) - } - case _ => fail("Unsupported type") - } - } - } - - test(s"$PassThrough with $typeName: empty column") { - skeleton(Seq.empty) - } - - test(s"$PassThrough with $typeName: long random series") { - val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeleton(input.map(_.asInstanceOf[T#InternalType])) - } - - test(s"$PassThrough with $typeName: empty column for decompress()") { - skeletonForDecompress(Seq.empty) - } - - test(s"$PassThrough with $typeName: long random series for decompress()") { - val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) - } - - test(s"$PassThrough with $typeName: simple case with null for decompress()") { - val input = columnType match { - case BYTE => Seq(2: Byte, 1: Byte, 2: Byte, nullValue.toByte: Byte, 5: Byte) - case SHORT => Seq(2: Short, 1: Short, 2: Short, nullValue.toShort: Short, 5: Short) - case INT => Seq(2: Int, 1: Int, 2: Int, nullValue: Int, 5: Int) - case LONG => Seq(2: Long, 1: Long, 2: Long, nullValue: Long, 5: Long) - case FLOAT => Seq(2: Float, 1: Float, 2: Float, nullValue: Float, 5: Float) - case DOUBLE => Seq(2: Double, 1: Double, 2: Double, nullValue: Double, 5: Double) - } - - skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) - } - } -} +/* + * 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.execution.columnar.compression + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.types.AtomicType + +class PassThroughSuite extends SparkFunSuite { + val nullValue = -1 + testPassThrough(new ByteColumnStats, BYTE) + testPassThrough(new ShortColumnStats, SHORT) + testPassThrough(new IntColumnStats, INT) + testPassThrough(new LongColumnStats, LONG) + testPassThrough(new FloatColumnStats, FLOAT) + testPassThrough(new DoubleColumnStats, DOUBLE) + + def testPassThrough[T <: AtomicType]( + columnStats: ColumnStats, + columnType: NativeColumnType[T]) { + + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + def skeleton(input: Seq[T#InternalType]) { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) + + input.map { value => + val row = new GenericInternalRow(1) + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + + val buffer = builder.build() + // Column type ID + null count + null positions + val headerSize = CompressionScheme.columnHeaderSize(buffer) + + // Compression scheme ID + compressed contents + val compressedSize = 4 + input.size * columnType.defaultSize + + // 4 extra bytes for compression scheme type ID + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + buffer.position(headerSize) + assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + if (input.nonEmpty) { + input.foreach { value => + assertResult(value, "Wrong value")(columnType.extract(buffer)) + } + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = PassThrough.decoder(buffer, columnType) + val mutableRow = new GenericInternalRow(1) + + if (input.nonEmpty) { + input.foreach{ + assert(decoder.hasNext) + assertResult(_, "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } + } + } + assert(!decoder.hasNext) + } + + def skeletonForDecompress(input: Seq[T#InternalType]) { + val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) + val row = new GenericInternalRow(1) + val nullRow = new GenericInternalRow(1) + nullRow.setNullAt(0) + input.map { value => + if (value == nullValue) { + builder.appendFrom(nullRow, 0) + } else { + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + } + val buffer = builder.build() + + // ---------------- + // Tests decompress + // ---------------- + // Rewinds, skips column header and 4 more bytes for compression scheme ID + val headerSize = CompressionScheme.columnHeaderSize(buffer) + buffer.position(headerSize) + assertResult(PassThrough.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val decoder = PassThrough.decoder(buffer, columnType) + val columnVector = new OnHeapColumnVector(input.length, columnType.dataType) + decoder.decompress(columnVector, input.length) + + if (input.nonEmpty) { + input.zipWithIndex.foreach { + case (expected: Any, index: Int) if expected == nullValue => + assertResult(true, s"Wrong null ${index}th-position") { + columnVector.isNullAt(index) + } + case (expected: Byte, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded byte value") { + columnVector.getByte(index) + } + case (expected: Short, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded short value") { + columnVector.getShort(index) + } + case (expected: Int, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded int value") { + columnVector.getInt(index) + } + case (expected: Long, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded long value") { + columnVector.getLong(index) + } + case (expected: Float, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded float value") { + columnVector.getFloat(index) + } + case (expected: Double, index: Int) => + assertResult(expected, s"Wrong ${index}-th decoded double value") { + columnVector.getDouble(index) + } + case _ => fail("Unsupported type") + } + } + } + + test(s"$PassThrough with $typeName: empty column") { + skeleton(Seq.empty) + } + + test(s"$PassThrough with $typeName: long random series") { + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeleton(input.map(_.asInstanceOf[T#InternalType])) + } + + test(s"$PassThrough with $typeName: empty column for decompress()") { + skeletonForDecompress(Seq.empty) + } + + test(s"$PassThrough with $typeName: long random series for decompress()") { + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) + } + + test(s"$PassThrough with $typeName: simple case with null for decompress()") { + val input = columnType match { + case BYTE => Seq(2: Byte, 1: Byte, 2: Byte, nullValue.toByte: Byte, 5: Byte) + case SHORT => Seq(2: Short, 1: Short, 2: Short, nullValue.toShort: Short, 5: Short) + case INT => Seq(2: Int, 1: Int, 2: Int, nullValue: Int, 5: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, nullValue: Long, 5: Long) + case FLOAT => Seq(2: Float, 1: Float, 2: Float, nullValue: Float, 5: Float) + case DOUBLE => Seq(2: Double, 1: Double, 2: Double, nullValue: Double, 5: Double) + } + + skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index 2263bb43b82b6..eb1cdd9bbceff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -34,9 +34,9 @@ class RunLengthEncodingSuite extends SparkFunSuite { testRunLengthEncoding(new StringColumnStats, STRING, false) def testRunLengthEncoding[T <: AtomicType]( - columnStats: ColumnStats, - columnType: NativeColumnType[T], - testDecompress: Boolean = true) { + columnStats: ColumnStats, + columnType: NativeColumnType[T], + testDecompress: Boolean = true) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$")