From f0a006387295298b830fa3fbea205e49b3d2e00c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:17:38 -0400 Subject: [PATCH 01/50] add a property 'spark.sql.inMemoryColumnarStorage.codegen' to controll code generation --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1a9bb6a0b54e..8038a997b549 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -84,6 +84,10 @@ object SQLConf { .intConf .createWithDefault(10000) + val COLUMN_VECTOR_CODEGEN = booleanConf("spark.sql.inMemoryColumnarStorage.codegen", + defaultValue = Some(true), + doc = "When set to true Spark SQL will generate code to access columnar storage.") + val IN_MEMORY_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.partitionPruning") .internal() From 8368eb5cb832bcab0ab584700fdf958ef487695f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:22:45 -0400 Subject: [PATCH 02/50] add CachedBatch.column() to prepare a columnar storage that is accessed by generated code --- .../execution/columnar/InMemoryRelation.scala | 30 ++++++++++++++++++- .../columnar/InMemoryTableScanExec.scala | 18 ++++++----- 2 files changed, 40 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 079e122a5a85..a1c64e9bfdb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -53,7 +53,35 @@ private[sql] object InMemoryRelation { * @param stats The stat of columns */ private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], + dataTypes: Array[DataType], stats: InternalRow) { + def column(ordinal: Int): org.apache.spark.sql.execution.vectorized.ColumnVector = { + val dt = dataTypes(ordinal) + val buffer = ByteBuffer.wrap(buffers(ordinal)).order(nativeOrder) + val accessor: BasicColumnAccessor[_] = dt match { + case BooleanType => new BooleanColumnAccessor(buffer) + case ByteType => new ByteColumnAccessor(buffer) + case ShortType => new ShortColumnAccessor(buffer) + case IntegerType | DateType => new IntColumnAccessor(buffer) + case LongType | TimestampType => new LongColumnAccessor(buffer) + case FloatType => new FloatColumnAccessor(buffer) + case DoubleType => new DoubleColumnAccessor(buffer) + } + + val (out, nullsBuffer) = if (accessor.isInstanceOf[NativeColumnAccessor[_]]) { + val nativeAccessor = accessor.asInstanceOf[NativeColumnAccessor[_]] + nativeAccessor.decompress(numRows); + } else { + val buffer = accessor.getByteBuffer + val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) + nullsBuffer.rewind() + (buffer, nullsBuffer) + } + + org.apache.spark.sql.execution.vectorized.ColumnVector.allocate(numRows, dt, + true, out, nullsBuffer) + } +} private[sql] case class InMemoryRelation( output: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 67a410f539b6..576cd61734bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.UserDefinedType +import org.apache.spark.sql.types._ private[sql] case class InMemoryTableScanExec( @@ -133,9 +133,9 @@ private[sql] case class InMemoryTableScanExec( schema) // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = + val (requestedColumnIndices, requestedColumnAttribute) = attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType + relOutput.indexOf(a.exprId) -> a }.unzip // Do partition batch pruning if enabled @@ -167,12 +167,16 @@ private[sql] case class InMemoryTableScanExec( batch } - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other + val columnTypes = requestedColumnAttribute.map { a => + a.dataType match { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + } }.toArray + val columnNullables = requestedColumnAttribute.map { _.nullable }.toArray val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) + columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray, + columnNullables) if (enableAccumulators && columnarIterator.hasNext) { readPartitions.add(1) } From 11659fc01e21fcc0e4d7c7c3d290e442f35cfeb8 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:26:20 -0400 Subject: [PATCH 03/50] add utility methods for putting a primitive value to ByteBuffer --- .../sql/execution/columnar/ColumnType.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) 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 f9d606e37ea8..5ae001457022 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 @@ -66,6 +66,24 @@ 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) + } } /** From 8d77142320fdb19fcf79adc6587c7f2d660cb7f6 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:32:32 -0400 Subject: [PATCH 04/50] add ByteBufferColumnVector, which wraps ByteByffer for columnar storage, as an implementation of ColumnVector --- .../vectorized/ByteBufferColumnVector.java | 399 ++++++++++++++++++ .../execution/vectorized/ColumnVector.java | 6 + 2 files changed, 405 insertions(+) create mode 100755 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java new file mode 100755 index 000000000000..78fc926e22e7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java @@ -0,0 +1,399 @@ +/* + * 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 java.util.Arrays; + +import org.apache.commons.lang.NotImplementedException; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow; +import org.apache.spark.sql.catalyst.expressions.MutableRow; +import org.apache.spark.sql.execution.columnar.BasicColumnAccessor; +import org.apache.spark.sql.execution.columnar.ByteBufferHelper; +import org.apache.spark.sql.execution.columnar.NativeColumnAccessor; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.Platform; + +/** + * A column backed by an in memory JVM array. This stores the NULLs as a byte per value + * and a java array for the values. + */ +public final class ByteBufferColumnVector extends ColumnVector { + // The data stored in these arrays need to maintain binary compatible. We can + // directly pass this buffer to external components. + + // This is faster than a boolean array and we optimize this over memory footprint. + private byte[] nulls; + + // Array for each type. Only 1 is populated for any type. + private byte[] data; + private long offset; + + protected ByteBufferColumnVector(int capacity, DataType type, + boolean isConstant, ByteBuffer buffer, ByteBuffer nullsBuffer) { + super(capacity, type, MemoryMode.ON_HEAP); + if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { + throw new NotImplementedException(); + } else if (type instanceof BooleanType) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (type instanceof ByteType) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (type instanceof ShortType) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (type instanceof IntegerType || type instanceof DateType || + DecimalType.is32BitDecimalType(type)) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (type instanceof LongType || DecimalType.is64BitDecimalType(type)) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (type instanceof FloatType) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (type instanceof DoubleType) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (resultStruct != null) { + // Nothing to store. + } else { + throw new RuntimeException("Unhandled " + type); + } + nulls = new byte[capacity]; + reset(); + + int numNulls = ByteBufferHelper.getInt(nullsBuffer); + for (int i = 0; i < numNulls; i++) { + int cordinal = ByteBufferHelper.getInt(nullsBuffer); + putNull(cordinal); + } + if (isConstant) { + setIsConstant(); + } + } + + @Override + public final long valuesNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + @Override + public final long nullsNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + + @Override + public final void close() { + } + + // + // APIs dealing with nulls + // + + @Override + public final void putNotNull(int rowId) { + nulls[rowId] = (byte)0; + } + + @Override + public final void putNull(int rowId) { + nulls[rowId] = (byte)1; + ++numNulls; + anyNullsSet = true; + } + + @Override + public final void putNulls(int rowId, int count) { + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)1; + } + anyNullsSet = true; + numNulls += count; + } + + @Override + public final void putNotNulls(int rowId, int count) { + if (!anyNullsSet) return; + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)0; + } + } + + @Override + public final boolean isNullAt(int rowId) { + System.out.println("isNullAt("+rowId+")="+nulls[rowId]); + return nulls[rowId] == 1; + } + + // + // APIs dealing with Booleans + // + + @Override + public final void putBoolean(int rowId, boolean value) { + Platform.putByte(data, offset + rowId, (byte)((value) ? 1 : 0)); + } + + @Override + public final void putBooleans(int rowId, int count, boolean value) { + byte v = (byte)((value) ? 1 : 0); + for (int i = 0; i < count; ++i) { + Platform.putByte(data, offset + i + rowId, v); + } + } + + @Override + public final boolean getBoolean(int rowId) { + return Platform.getByte(data, offset + rowId) == 1; + } + + // + + // + // APIs dealing with Bytes + // + + @Override + public final void putByte(int rowId, byte value) { + Platform.putByte(data, offset + rowId, value); + } + + @Override + public final void putBytes(int rowId, int count, byte value) { + for (int i = 0; i < count; ++i) { + Platform.putByte(data, offset + i + rowId, value); + } + } + + @Override + public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { + System.arraycopy(src, srcIndex, data, rowId, count); + } + + @Override + public final byte getByte(int rowId) { + if (dictionary == null) { + return Platform.getByte(data, offset + rowId); + } else { + return (byte) dictionary.decodeToInt(dictionaryIds.getInt(rowId)); + } + } + + // + // APIs dealing with Shorts + // + + @Override + public final void putShort(int rowId, short value) { + Platform.putShort(data, offset + rowId * 2, value); + } + + @Override + public final void putShorts(int rowId, int count, short value) { + for (int i = 0; i < count; ++i) { + Platform.putShort(data, offset + (i + rowId) * 2, value); + } + } + + @Override + public final void putShorts(int rowId, int count, short[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final short getShort(int rowId) { + if (dictionary == null) { + return Platform.getShort(data, offset + rowId * 2); + } else { + return (short) dictionary.decodeToInt(dictionaryIds.getInt(rowId)); + } + } + + + // + // APIs dealing with Ints + // + + @Override + public final void putInt(int rowId, int value) { + Platform.putInt(data, offset + rowId * 4, value); + } + + @Override + public final void putInts(int rowId, int count, int value) { + for (int i = 0; i < count; ++i) { + Platform.putInt(data, offset + (i + rowId) * 4, value); + } + } + + @Override + public final void putInts(int rowId, int count, int[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final int getInt(int rowId) { + if (dictionary == null) { + return Platform.getInt(data, offset + rowId * 4); + } else { + return dictionary.decodeToInt(dictionaryIds.getInt(rowId)); + } + } + + // + // APIs dealing with Longs + // + + @Override + public final void putLong(int rowId, long value) { + Platform.putLong(data, offset + rowId * 8, value); + } + + @Override + public final void putLongs(int rowId, int count, long value) { + for (int i = 0; i < count; ++i) { + Platform.getLong(data, offset + (i + rowId) * 8); + } + } + + @Override + public final void putLongs(int rowId, int count, long[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final long getLong(int rowId) { + if (dictionary == null) { + return Platform.getLong(data, offset + rowId * 8); + } else { + return dictionary.decodeToLong(dictionaryIds.getInt(rowId)); + } + } + + // + // APIs dealing with floats + // + + @Override + public final void putFloat(int rowId, float value) { Platform.putFloat(data, offset + rowId * 4, value); } + + @Override + public final void putFloats(int rowId, int count, float value) { + throw new NotImplementedException(); + } + + @Override + public final void putFloats(int rowId, int count, float[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + data, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + } + + @Override + public final float getFloat(int rowId) { + if (dictionary == null) { + return Platform.getFloat(data, offset + rowId * 4); + } else { + return dictionary.decodeToFloat(dictionaryIds.getInt(rowId)); + } + } + + // + // APIs dealing with doubles + // + + @Override + public final void putDouble(int rowId, double value) { Platform.putDouble(data, offset + rowId * 8, value); } + + @Override + public final void putDoubles(int rowId, int count, double value) { + throw new NotImplementedException(); + } + + @Override + public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final double getDouble(int rowId) { + if (dictionary == null) { + return Platform.getDouble(data, offset + rowId * 8); + } else { + return dictionary.decodeToDouble(dictionaryIds.getInt(rowId)); + } + } + + // + // APIs dealing with Arrays + // + + @Override + public final int getArrayLength(int rowId) { throw new NotImplementedException(); } + @Override + public final int getArrayOffset(int rowId) { throw new NotImplementedException(); } + + @Override + public final void putArray(int rowId, int offset, int length) { + throw new NotImplementedException(); + } + + @Override + public final void loadBytes(ColumnVector.Array array) { + throw new NotImplementedException(); + } + + // + // APIs dealing with Byte Arrays + // + + @Override + public final int putByteArray(int rowId, byte[] value, int offset, int length) { + throw new NotImplementedException(); + } + + @Override + public final void reserve(int requiredCapacity) { + if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); + } + + // Spilt this function out since it is the slow path. + private final void reserveInternal(int newCapacity) { + throw new NotImplementedException(); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index bbbb796aca0d..1ea34673dc4f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -18,6 +18,7 @@ import java.math.BigDecimal; import java.math.BigInteger; +import java.nio.ByteBuffer; import com.google.common.annotations.VisibleForTesting; import org.apache.parquet.column.Dictionary; @@ -71,6 +72,11 @@ public static ColumnVector allocate(int capacity, DataType type, MemoryMode mode } } + public static ColumnVector allocate(int capacity, DataType type, + boolean isConstant, ByteBuffer buffer, ByteBuffer nullsBuffer) { + return new ByteBufferColumnVector(capacity, type, isConstant, buffer, nullsBuffer); + } + /** * Holder object to return an array. This object is intended to be reused. Callers should * copy the data out if it needs to be stored. From 7d6ad334810cdc979af0f5ac2a1eeb2d748e1144 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:34:59 -0400 Subject: [PATCH 05/50] add APIs to get a CachedBatch --- .../columnar/GenerateColumnAccessor.scala | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 7a14879b8b9d..2ffc9934ae84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.types._ */ abstract class ColumnarIterator extends Iterator[InternalRow] { def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], - columnIndexes: Array[Int]): Unit + columnIndexes: Array[Int], columnNullables: Array[Boolean]): Unit + def getInput: Iterator[CachedBatch] + def getColumnIndexes: Array[Int] + def isSupportColumnarCodeGen: Boolean } /** @@ -68,6 +71,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera protected def create(columnTypes: Seq[DataType]): ColumnarIterator = { val ctx = newCodeGenContext() val numFields = columnTypes.size + var _isSupportColumnarCodeGen = true val (initializeAccessors, extractors) = columnTypes.zipWithIndex.map { case (dt, index) => val accessorName = ctx.freshName("accessor") val accessorCls = dt match { @@ -94,8 +98,10 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera case t if ctx.isPrimitiveType(dt) => s"$accessorName = new $accessorCls(ByteBuffer.wrap(buffers[$index]).order(nativeOrder));" case NullType | StringType | BinaryType => + _isSupportColumnarCodeGen = false s"$accessorName = new $accessorCls(ByteBuffer.wrap(buffers[$index]).order(nativeOrder));" case other => + _isSupportColumnarCodeGen = false s"""$accessorName = new $accessorCls(ByteBuffer.wrap(buffers[$index]).order(nativeOrder), (${dt.getClass.getName}) columnTypes[$index]);""" } @@ -178,6 +184,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera private scala.collection.Iterator input = null; private DataType[] columnTypes = null; private int[] columnIndexes = null; + private boolean[] columnNullables = null; ${ctx.declareMutableStates()} @@ -187,10 +194,12 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera this.mutableRow = new MutableUnsafeRow(rowWriter); } - public void initialize(Iterator input, DataType[] columnTypes, int[] columnIndexes) { + public void initialize(Iterator input, DataType[] columnTypes, int[] columnIndexes, + boolean columnNullables[]) { this.input = input; this.columnTypes = columnTypes; this.columnIndexes = columnIndexes; + this.columnNullables = columnNullables; } ${ctx.declareAddedFunctions()} @@ -222,6 +231,14 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera unsafeRow.setTotalSize(bufferHolder.totalSize()); return unsafeRow; } + + public scala.collection.Iterator getInput() { return input; } + + public int[] getColumnIndexes() { return columnIndexes; } + + public boolean isSupportColumnarCodeGen() { + return ${_isSupportColumnarCodeGen}; + } }""" val code = CodeFormatter.stripOverlappingComments( From df2c9a328153428e15cc2f35450ff6c6a4af121d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:36:10 -0400 Subject: [PATCH 06/50] add decompress() method --- .../CompressibleColumnAccessor.scala | 4 + .../compression/CompressionScheme.scala | 2 + .../compression/compressionSchemes.scala | 328 +++++++++++++++++- 3 files changed, 333 insertions(+), 1 deletion(-) mode change 100644 => 100755 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala 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 6579b5068e65..fa2f915e7221 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.expressions.MutableRow 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: MutableRow, 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 b90d00b15b18..cecb4d79885c 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 @@ -42,6 +42,8 @@ private[columnar] trait Decoder[T <: AtomicType] { def next(row: MutableRow, 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 old mode 100644 new mode 100755 index 941f03b745a0..b84c82933229 --- 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,12 @@ 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() + (buffer, nullsBuffer) + } } } @@ -164,11 +171,183 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCount += 1 } - columnType.setField(row, ordinal, currentValue) } 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 + 2) + } + 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 (valueCountLocal == runLocal) { + currentValueLocal = buffer.getInt() + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 + } else { + valueCountLocal += 1 + } + ByteBufferHelper.putInt(out, currentValueLocal) + } +*/ + + val inarray = buffer.array + var inoffset = buffer.position + val inlimit = buffer.limit + val outarray = out.array + var outoffset = 0 + while (inoffset < inlimit) { + val currentValueLocal = + org.apache.spark.unsafe.Platform.getInt(inarray, + org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET + + inoffset) + val runLocal0 = + org.apache.spark.unsafe.Platform.getInt(inarray, + org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET + + inoffset + 4) + inoffset += 8 + var i = 0 + while (i < runLocal0) { + if (pos != nextNullIndex) { + org.apache.spark.unsafe.Platform.putInt(outarray, + org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET + + outoffset, currentValueLocal) + i += 1 + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) + } + } + pos += 1 + outoffset += 4 + } + } + buffer.position(inoffset) + + 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 +457,56 @@ 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 +597,42 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { } override def hasNext: Boolean = visited < count + + override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { + val countLocal = ByteBufferHelper.getInt(buffer) + 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 (((currentWord >> 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 +713,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 +824,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 * 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.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) + } } } From dafaa7f4813bde10583f7d6002fd80dff83a19f1 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:36:49 -0400 Subject: [PATCH 07/50] add decompress() method --- .../apache/spark/sql/execution/columnar/ColumnAccessor.scala | 2 ++ 1 file changed, 2 insertions(+) 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 7cde04b62619..1b41cb380624 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 @@ -61,6 +61,8 @@ private[columnar] abstract class BasicColumnAccessor[JvmType]( } protected def underlyingBuffer = buffer + + def getByteBuffer: ByteBuffer = buffer } private[columnar] class NullColumnAccessor(buffer: ByteBuffer) From 0771188e6f41a061271e57421b634ed56185a9e9 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:37:20 -0400 Subject: [PATCH 08/50] add ColumnVectorReference class --- .../catalyst/expressions/BoundAttribute.scala | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 7d16118c9d59..a381c09230f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -100,3 +100,32 @@ object BindReferences extends Logging { }.asInstanceOf[A] // Kind of a hack, but safe. TODO: Tighten return type when possible. } } + +/** + * A column vector reference points to a specific column for ColumnVector. + * columnVar is a variable that keeps ColumnVector, and ordinal is row index in ColumnVector + */ +case class ColumnVectorReference( + columnVar: String, ordinal: String, dataType: DataType, nullable: Boolean) + extends LeafExpression { + + override def toString: String = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + + override def eval(input: InternalRow): Any = null + + override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + val javaType = ctx.javaType(dataType) + val value = ctx.getValue(columnVar, dataType, ordinal) + if (nullable) { + s""" + boolean ${ev.isNull} = ${columnVar}.isNullAt($ordinal); + $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : ($value); + """ + } else { + ev.isNull = "false" + s""" + $javaType ${ev.value} = $value; + """ + } + } +} From 47a88e57325bcb5146d274cc944c3702e3b033cd Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:38:52 -0400 Subject: [PATCH 09/50] Do not call createHashMap() until actually executed at first --- .../apache/spark/sql/execution/aggregate/HashAggregateExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 54d7340d8acd..81cbccd8e5a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -576,6 +576,7 @@ case class HashAggregateExec( if (!$initAgg) { $initAgg = true; long $beforeAgg = System.nanoTime(); + $hashMapTerm = $thisPlan.createHashMap(); $doAgg(); $aggTime.add((System.nanoTime() - $beforeAgg) / 1000000); } From 2b5bc2f9e4aa6fefcf6026ef0355c9e4f35a166b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:40:45 -0400 Subject: [PATCH 10/50] generate two version of Java codes for row-oriented and column-oriented storages if InMemoryColumnar exists in a tree plan --- .../expressions/codegen/CodeGenerator.scala | 20 ++++++-- .../sql/execution/WholeStageCodegenExec.scala | 51 +++++++++++++++++-- .../execution/basicPhysicalOperators.scala | 1 + 3 files changed, 63 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 6392ff42d709..2bb35c4715bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -123,11 +123,12 @@ class CodegenContext { * * They will be kept as member variables in generated classes like `SpecificProjection`. */ - val mutableStates: mutable.ArrayBuffer[(String, String, String)] = - mutable.ArrayBuffer.empty[(String, String, String)] + val mutableStates: mutable.ArrayBuffer[(String, String, String, String)] = + mutable.ArrayBuffer.empty[(String, String, String, String)] - def addMutableState(javaType: String, variableName: String, initCode: String): Unit = { - mutableStates += ((javaType, variableName, initCode)) + def addMutableState(javaType: String, variableName: String, initCode: String, + cleanupCode: String = ""): Unit = { + mutableStates += ((javaType, variableName, initCode, cleanupCode)) } /** @@ -149,7 +150,7 @@ class CodegenContext { def declareMutableStates(): String = { // It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in // `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones. - mutableStates.distinct.map { case (javaType, variableName, _) => + mutableStates.distinct.map { case (javaType, variableName, _, _) => s"private $javaType $variableName;" }.mkString("\n") } @@ -160,6 +161,9 @@ class CodegenContext { mutableStates.distinct.map(_._3).mkString("\n") } + def cleanupMutableStates(): String = { + mutableStates.map(_._4).mkString("\n") + } /** * Holding all the functions those will be added into generated class. */ @@ -206,6 +210,12 @@ class CodegenContext { /** The variable name of the input row in generated code. */ final var INPUT_ROW = "i" + var isRow = true + var enableColumnCodeGen = false + var inputHolder = "" + val columnarBatchName = "columnar_batch" + val columnarBatchIdxName = "columnar_batchIdx" + /** * The map from a variable name to it's next ID. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index ac4c3aae5f8e..39cc5826d82a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -24,7 +24,10 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.execution.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf @@ -234,10 +237,19 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp } override def doProduce(ctx: CodegenContext): String = { + ctx.enableColumnCodeGen = true val input = ctx.freshName("input") + ctx.inputHolder = input // Right now, InputAdapter is only used when there is one input RDD. ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") - val row = ctx.freshName("row") + + if (ctx.isRow) { + val exprRows = output.zipWithIndex.map(x => + new BoundReference(x._2, x._1.dataType, x._1.nullable)) + val row = ctx.freshName("row") + ctx.INPUT_ROW = row + ctx.currentVars = null + val columns = exprRows.map(_.gen(ctx)) s""" | while ($input.hasNext()) { | InternalRow $row = (InternalRow) $input.next(); @@ -245,6 +257,39 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | if (shouldStop()) return; | } """.stripMargin + } else { + val columnarBatchClz = "org.apache.spark.sql.execution.columnar.CachedBatch" + val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" + val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" + val batch = ctx.columnarBatchName + val idx = ctx.columnarBatchIdxName + val rowidx = ctx.freshName("rowIdx") + val col = ctx.freshName("col") + val numrows = ctx.freshName("numRows") + val colVars = output.indices.map(i => ctx.freshName("col" + i)) + val columnAssigns = colVars.zipWithIndex.map { case (name, i) => + ctx.addMutableState(columnVectorClz, name, s"$name = null;", s"$name = null;") + s"$name = batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } + + ctx.currentVars = null + val exprCols = (output zip colVars).map { case (attr, colVar) => + new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable) } + val columns = exprCols.map(_.gen(ctx)) + s""" + | org.apache.spark.sql.execution.columnar.CachedBatch batch = + | (org.apache.spark.sql.execution.columnar.CachedBatch) $batch; + | + | if ($idx == 0) { + | ${columnAssigns.mkString("", "\n", "")} + | } + | + | int $numrows = batch.numRows(); + | while (!shouldStop() && ($idx < $numrows)) { + | int $rowidx = $idx++; + | ${consume(ctx, columns, col).trim} + | } + """.stripMargin + } } override def generateTreeString( @@ -329,9 +374,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co ${ctx.declareAddedFunctions()} - protected void processNext() throws java.io.IOException { - ${code.trim} - } + ${codeProcess} } """.trim diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 185c79f899e6..dfad3c2066e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -382,6 +382,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) """.stripMargin) val input = ctx.freshName("input") + ctx.inputHolder = input // Right now, Range is only used when there is one upstream. ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") s""" From 1f1e685d14c246cc0a67db34e05b5985d011fc92 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 04:58:29 -0400 Subject: [PATCH 11/50] apply SPARK-14092 to generated code for columnar storage --- .../sql/execution/WholeStageCodegenExec.scala | 77 ++++++++++++++++++- 1 file changed, 75 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 39cc5826d82a..92bc44c2fece 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -284,9 +284,10 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | } | | int $numrows = batch.numRows(); - | while (!shouldStop() && ($idx < $numrows)) { + | while ($idx < $numrows)) { | int $rowidx = $idx++; | ${consume(ctx, columns, col).trim} + | if (shouldStop()) return; | } """.stripMargin } @@ -351,7 +352,79 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co */ def doCodeGen(): (CodegenContext, CodeAndComment) = { val ctx = new CodegenContext - val code = child.asInstanceOf[CodegenSupport].produce(ctx, this) + ctx.isRow = true + val codeRow = child.asInstanceOf[CodegenSupport].produce(ctx, this) + val referUnsafeRow = child.find(c => c.isInstanceOf[CodegenSupport] && + c.asInstanceOf[CodegenSupport].preferUnsafeRow) match { + case Some(c) => true + case None => false + } + val useInMemoryColumnar = child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]) match { + case Some(c) => true + case None => false + } + val enableColumnCodeGen = ctx.enableColumnCodeGen && !referUnsafeRow && useInMemoryColumnar && + sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean + val codeCol = if (enableColumnCodeGen) { + ctx.isRow = false + child.asInstanceOf[CodegenSupport].produce(ctx, this) + } else null + + ctx.addMutableState("Object", ctx.columnarBatchName, s"${ctx.columnarBatchName} = null;") + ctx.addMutableState("int", ctx.columnarBatchIdxName, s"${ctx.columnarBatchIdxName} = 0;") + + val codeProcess = if (codeCol == null) { + s""" + protected void processNext() throws java.io.IOException { + ${codeRow.trim} + } + """ + } else { + s""" + private void processBatch(scala.collection.Iterator itr) throws java.io.IOException { + while (true) { + if (${ctx.columnarBatchIdxName} == 0) { + if (itr.hasNext()) { + ${ctx.columnarBatchName} = itr.next(); + } else { + cleanup(); + return; + } + } + ${codeCol.trim} + + ${ctx.columnarBatchIdxName} = 0; + } + } + + private void processRow() throws java.io.IOException { + ${codeRow.trim} + } + + private void cleanup() { + ${ctx.columnarBatchName} = null; + ${ctx.cleanupMutableStates()} + } + + protected void processNext() throws java.io.IOException { + org.apache.spark.sql.execution.columnar.ColumnarIterator columnItr = null; + if (${ctx.columnarBatchName} != null) { + columnItr = (org.apache.spark.sql.execution.columnar.ColumnarIterator) + ${ctx.inputHolder}; + processBatch(columnItr.getInput()); + } else if (${ctx.inputHolder} instanceof + org.apache.spark.sql.execution.columnar.ColumnarIterator && + ((columnItr = (org.apache.spark.sql.execution.columnar.ColumnarIterator) + ${ctx.inputHolder}).isSupportColumnarCodeGen())) { + ${ctx.columnarBatchIdxName} = 0; + processBatch(columnItr.getInput()); + } else { + processRow(); + } + } + """.trim + } + val source = s""" public Object generate(Object[] references) { return new GeneratedIterator(references); From 568dc4769efe7abee32239a7f4ee904a444e7ca0 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 06:04:13 -0400 Subject: [PATCH 12/50] fix compilation error at runtime --- .../org/apache/spark/sql/execution/WholeStageCodegenExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 92bc44c2fece..6ea6dbe943d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -284,7 +284,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | } | | int $numrows = batch.numRows(); - | while ($idx < $numrows)) { + | while ($idx < $numrows) { | int $rowidx = $idx++; | ${consume(ctx, columns, col).trim} | if (shouldStop()) return; From 5b5f037f4f2452a213abc63ed9480b624a68fde1 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 25 Mar 2016 08:11:15 -0400 Subject: [PATCH 13/50] drop debug print --- .../spark/sql/execution/vectorized/ByteBufferColumnVector.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java index 78fc926e22e7..48f1d55168f3 100755 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java @@ -137,7 +137,6 @@ public final void putNotNulls(int rowId, int count) { @Override public final boolean isNullAt(int rowId) { - System.out.println("isNullAt("+rowId+")="+nulls[rowId]); return nulls[rowId] == 1; } From 6542af7a9559195e71584f147f2cf7d97abc9156 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 30 Mar 2016 23:10:11 -0400 Subject: [PATCH 14/50] merge with the latest --- .../apache/spark/sql/execution/SortExec.scala | 2 ++ .../sql/execution/WholeStageCodegenExec.scala | 18 +++++++++--------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 6db7f45cfdf2..f9aac8bcc587 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -122,6 +122,8 @@ case class SortExec( // Name of sorter variable used in codegen. private var sorterVariable: String = _ + override def preferUnsafeRow: Boolean = true + override protected def doProduce(ctx: CodegenContext): String = { val needToSort = ctx.freshName("needToSort") ctx.addMutableState("boolean", needToSort, s"$needToSort = true;") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 6ea6dbe943d8..269082cca0d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -68,6 +68,11 @@ trait CodegenSupport extends SparkPlan { */ protected var parent: CodegenSupport = null + /** + * Whether this SparkPlan prefers to accept UnsafeRow as input in doConsume + */ + def preferUnsafeRow: Boolean = false + /** * Returns all the RDDs of InternalRow which generates the input rows. * @@ -113,6 +118,7 @@ trait CodegenSupport extends SparkPlan { final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { val inputVars = if (row != null) { + assert(ctx.isRow) ctx.currentVars = null ctx.INPUT_ROW = row output.zipWithIndex.map { case (attr, i) => @@ -244,12 +250,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") if (ctx.isRow) { - val exprRows = output.zipWithIndex.map(x => - new BoundReference(x._2, x._1.dataType, x._1.nullable)) val row = ctx.freshName("row") - ctx.INPUT_ROW = row - ctx.currentVars = null - val columns = exprRows.map(_.gen(ctx)) s""" | while ($input.hasNext()) { | InternalRow $row = (InternalRow) $input.next(); @@ -272,9 +273,8 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp s"$name = batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } ctx.currentVars = null - val exprCols = (output zip colVars).map { case (attr, colVar) => - new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable) } - val columns = exprCols.map(_.gen(ctx)) + val columns = (output zip colVars).map { case (attr, colVar) => + new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).gen(ctx) } s""" | org.apache.spark.sql.execution.columnar.CachedBatch batch = | (org.apache.spark.sql.execution.columnar.CachedBatch) $batch; @@ -286,7 +286,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | int $numrows = batch.numRows(); | while ($idx < $numrows) { | int $rowidx = $idx++; - | ${consume(ctx, columns, col).trim} + | ${consume(ctx, columns, null).trim} | if (shouldStop()) return; | } """.stripMargin From c47dac813ec215493aee630ef961e63f9ab31a68 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 10 Apr 2016 15:21:53 -0400 Subject: [PATCH 15/50] rebase --- .../spark/sql/execution/WholeStageCodegenExec.scala | 4 ++-- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 269082cca0d0..8b8bbcf5bcc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -69,8 +69,8 @@ trait CodegenSupport extends SparkPlan { protected var parent: CodegenSupport = null /** - * Whether this SparkPlan prefers to accept UnsafeRow as input in doConsume - */ + * Whether this SparkPlan prefers to accept UnsafeRow as input in doConsume + */ def preferUnsafeRow: Boolean = false /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8038a997b549..5b9df54364f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -84,9 +84,11 @@ object SQLConf { .intConf .createWithDefault(10000) - val COLUMN_VECTOR_CODEGEN = booleanConf("spark.sql.inMemoryColumnarStorage.codegen", - defaultValue = Some(true), - doc = "When set to true Spark SQL will generate code to access columnar storage.") + val COLUMN_VECTOR_CODEGEN = SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.codegen") + .internal() + .doc("When set to true Spark SQL will generate code to access columnar storage.") + .booleanConf + .createWithDefault(true) val IN_MEMORY_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.partitionPruning") From 55d3330b70dab0eaf631b91756aab9fea221d4e5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 13 Apr 2016 04:57:47 +0900 Subject: [PATCH 16/50] fix scalastyle error --- .../compression/compressionSchemes.scala | 16 +++++----------- 1 file changed, 5 insertions(+), 11 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 b84c82933229..560459e30212 100755 --- 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 @@ -186,7 +186,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { var runLocal = 0 var valueCountLocal = 0 columnType.dataType match { - case _: BooleanType => { + case _: BooleanType => val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) var currentValueLocal: Boolean = false while (valueCountLocal < runLocal || buffer.hasRemaining) { @@ -211,8 +211,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } - case _: ByteType => { + case _: ByteType => val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) var currentValueLocal: Byte = 0 while (valueCountLocal < runLocal || buffer.hasRemaining) { @@ -237,8 +236,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } - case _: ShortType => { + case _: ShortType => val out = ByteBuffer.allocate(capacity * 2).order(ByteOrder.nativeOrder()) var currentValueLocal: Short = 0 while (valueCountLocal < runLocal || buffer.hasRemaining) { @@ -263,8 +261,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } - case _: IntegerType => { + case _: IntegerType => val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) /* var currentValueLocal: Int = 0 @@ -279,7 +276,6 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { ByteBufferHelper.putInt(out, currentValueLocal) } */ - val inarray = buffer.array var inoffset = buffer.position val inlimit = buffer.limit @@ -317,8 +313,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } - case _: LongType => { + case _: LongType => val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) var currentValueLocal: Long = 0 @@ -344,7 +339,6 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } case _ => throw new IllegalStateException("Not supported type in RunLengthEncoding.") } } From 7bef54ffda99be2ad07a98a4150b9061b229b224 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 13 Apr 2016 05:17:22 +0900 Subject: [PATCH 17/50] fix scalastyle error --- .../execution/columnar/compression/compressionSchemes.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 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 560459e30212..4f7dd2d7997f 100755 --- 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 @@ -460,7 +460,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { var pos = 0 var seenNulls = 0 columnType.dataType match { - case _: IntegerType => { + case _: IntegerType => val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) while (buffer.hasRemaining) { if (pos != nextNullIndex) { @@ -478,8 +478,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } - case _: LongType => { + case _: LongType => val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) while (buffer.hasRemaining) { if (pos != nextNullIndex) { @@ -497,7 +496,6 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { out.rewind() nullsBuffer.rewind() (out, nullsBuffer) - } case _ => throw new IllegalStateException("Not supported type in DictionaryEncoding.") } } From 0d62d64f8d87cc6c49c8aa99b707717be3e2d78d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Apr 2016 01:20:57 +0900 Subject: [PATCH 18/50] make source code shorter --- .../org/apache/spark/sql/execution/WholeStageCodegenExec.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 8b8bbcf5bcc9..4105cce26acf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -276,8 +276,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).gen(ctx) } s""" - | org.apache.spark.sql.execution.columnar.CachedBatch batch = - | (org.apache.spark.sql.execution.columnar.CachedBatch) $batch; + | $columnarBatchClz batch = ($columnarBatchClz) $batch; | | if ($idx == 0) { | ${columnAssigns.mkString("", "\n", "")} From 7c062220050dcd2f7b07ca74464c80f1440a8d7b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Apr 2016 01:22:57 +0900 Subject: [PATCH 19/50] avoid memory leak due to twice allocations of hashMap --- .../apache/spark/sql/execution/aggregate/HashAggregateExec.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 81cbccd8e5a3..54d7340d8acd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -576,7 +576,6 @@ case class HashAggregateExec( if (!$initAgg) { $initAgg = true; long $beforeAgg = System.nanoTime(); - $hashMapTerm = $thisPlan.createHashMap(); $doAgg(); $aggTime.add((System.nanoTime() - $beforeAgg) / 1000000); } From 36af6ab811f819b8376ea2e83c65ba98f6e0a6da Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Apr 2016 03:28:05 +0900 Subject: [PATCH 20/50] fix assertion error --- .../org/apache/spark/sql/execution/WholeStageCodegenExec.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 4105cce26acf..f3e54ed23595 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -118,7 +118,6 @@ trait CodegenSupport extends SparkPlan { final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { val inputVars = if (row != null) { - assert(ctx.isRow) ctx.currentVars = null ctx.INPUT_ROW = row output.zipWithIndex.map { case (attr, i) => From eef56e4a6098df1671fcf89b81405ab369dfcdf9 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 14 Apr 2016 15:12:01 -0400 Subject: [PATCH 21/50] make decompress() simple --- .../compression/compressionSchemes.scala | 54 +++++-------------- 1 file changed, 13 insertions(+), 41 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 4f7dd2d7997f..be430bececbb 100755 --- 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 @@ -263,53 +263,25 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { (out, nullsBuffer) case _: IntegerType => val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) -/* var currentValueLocal: Int = 0 while (valueCountLocal < runLocal || buffer.hasRemaining) { - if (valueCountLocal == runLocal) { - currentValueLocal = buffer.getInt() - runLocal = ByteBufferHelper.getInt(buffer) - valueCountLocal = 1 - } else { - valueCountLocal += 1 - } - ByteBufferHelper.putInt(out, currentValueLocal) - } -*/ - val inarray = buffer.array - var inoffset = buffer.position - val inlimit = buffer.limit - val outarray = out.array - var outoffset = 0 - while (inoffset < inlimit) { - val currentValueLocal = - org.apache.spark.unsafe.Platform.getInt(inarray, - org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET + - inoffset) - val runLocal0 = - org.apache.spark.unsafe.Platform.getInt(inarray, - org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET + - inoffset + 4) - inoffset += 8 - var i = 0 - while (i < runLocal0) { - if (pos != nextNullIndex) { - org.apache.spark.unsafe.Platform.putInt(outarray, - org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET + - outoffset, currentValueLocal) - i += 1 + if (pos != nextNullIndex) { + if (valueCountLocal == runLocal) { + currentValueLocal = buffer.getInt() + runLocal = ByteBufferHelper.getInt(buffer) + valueCountLocal = 1 } else { - seenNulls += 1 - if (seenNulls < nullCount) { - nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) - } + valueCountLocal += 1 + } + ByteBufferHelper.putInt(out, currentValueLocal) + } else { + seenNulls += 1 + if (seenNulls < nullCount) { + nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - pos += 1 - outoffset += 4 + out.position(out.position + 4) } } - buffer.position(inoffset) - out.rewind() nullsBuffer.rewind() (out, nullsBuffer) From 6b47fb441e3ab85e5534b677623c5b41ff744a6f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 16 Apr 2016 15:14:12 -0400 Subject: [PATCH 22/50] support nulls in decompress() for PassThru --- .../sql/execution/columnar/ColumnType.scala | 9 ++++ .../compression/compressionSchemes.scala | 49 +++++++++++++++++-- 2 files changed, 53 insertions(+), 5 deletions(-) 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 5ae001457022..718dc6c5d406 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 @@ -84,6 +84,15 @@ private[columnar] object ByteBufferHelper { 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/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index be430bececbb..84bf839bda56 100755 --- 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 @@ -66,7 +66,46 @@ private[columnar] case object PassThrough extends CompressionScheme { override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind() - (buffer, nullsBuffer) + val nullCount = ByteBufferHelper.getInt(nullsBuffer) + if (nullCount == 0) { + nullsBuffer.rewind() + (buffer, 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) + } } } } @@ -178,7 +217,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) - nullsBuffer.rewind + nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 var pos = 0 @@ -426,7 +465,7 @@ private[columnar] case object DictionaryEncoding extends CompressionScheme { override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) - nullsBuffer.rewind + nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 var pos = 0 @@ -568,7 +607,7 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { var visitedLocal: Int = 0 val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) - nullsBuffer.rewind + nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 var pos = 0 @@ -682,7 +721,7 @@ private[columnar] case object IntDelta extends CompressionScheme { var prevLocal: Int = 0 val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) - nullsBuffer.rewind + nullsBuffer.rewind() val nullCount = ByteBufferHelper.getInt(nullsBuffer) var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1 var pos = 0 From 9f30d2f312c31dadf221dd5710c9edd4a737c79f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 16 Apr 2016 15:14:42 -0400 Subject: [PATCH 23/50] make it simple --- .../vectorized/ByteBufferColumnVector.java | 764 +++++++++--------- 1 file changed, 366 insertions(+), 398 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java index 48f1d55168f3..e385f6e4a281 100755 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java @@ -1,398 +1,366 @@ -/* - * 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 java.util.Arrays; - -import org.apache.commons.lang.NotImplementedException; -import org.apache.spark.memory.MemoryMode; -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow; -import org.apache.spark.sql.catalyst.expressions.MutableRow; -import org.apache.spark.sql.execution.columnar.BasicColumnAccessor; -import org.apache.spark.sql.execution.columnar.ByteBufferHelper; -import org.apache.spark.sql.execution.columnar.NativeColumnAccessor; -import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.Platform; - -/** - * A column backed by an in memory JVM array. This stores the NULLs as a byte per value - * and a java array for the values. - */ -public final class ByteBufferColumnVector extends ColumnVector { - // The data stored in these arrays need to maintain binary compatible. We can - // directly pass this buffer to external components. - - // This is faster than a boolean array and we optimize this over memory footprint. - private byte[] nulls; - - // Array for each type. Only 1 is populated for any type. - private byte[] data; - private long offset; - - protected ByteBufferColumnVector(int capacity, DataType type, - boolean isConstant, ByteBuffer buffer, ByteBuffer nullsBuffer) { - super(capacity, type, MemoryMode.ON_HEAP); - if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { - throw new NotImplementedException(); - } else if (type instanceof BooleanType) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (type instanceof ByteType) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (type instanceof ShortType) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (type instanceof IntegerType || type instanceof DateType || - DecimalType.is32BitDecimalType(type)) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (type instanceof LongType || DecimalType.is64BitDecimalType(type)) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (type instanceof FloatType) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (type instanceof DoubleType) { - data = buffer.array(); - offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); - } else if (resultStruct != null) { - // Nothing to store. - } else { - throw new RuntimeException("Unhandled " + type); - } - nulls = new byte[capacity]; - reset(); - - int numNulls = ByteBufferHelper.getInt(nullsBuffer); - for (int i = 0; i < numNulls; i++) { - int cordinal = ByteBufferHelper.getInt(nullsBuffer); - putNull(cordinal); - } - if (isConstant) { - setIsConstant(); - } - } - - @Override - public final long valuesNativeAddress() { - throw new RuntimeException("Cannot get native address for on heap column"); - } - @Override - public final long nullsNativeAddress() { - throw new RuntimeException("Cannot get native address for on heap column"); - } - - @Override - public final void close() { - } - - // - // APIs dealing with nulls - // - - @Override - public final void putNotNull(int rowId) { - nulls[rowId] = (byte)0; - } - - @Override - public final void putNull(int rowId) { - nulls[rowId] = (byte)1; - ++numNulls; - anyNullsSet = true; - } - - @Override - public final void putNulls(int rowId, int count) { - for (int i = 0; i < count; ++i) { - nulls[rowId + i] = (byte)1; - } - anyNullsSet = true; - numNulls += count; - } - - @Override - public final void putNotNulls(int rowId, int count) { - if (!anyNullsSet) return; - for (int i = 0; i < count; ++i) { - nulls[rowId + i] = (byte)0; - } - } - - @Override - public final boolean isNullAt(int rowId) { - return nulls[rowId] == 1; - } - - // - // APIs dealing with Booleans - // - - @Override - public final void putBoolean(int rowId, boolean value) { - Platform.putByte(data, offset + rowId, (byte)((value) ? 1 : 0)); - } - - @Override - public final void putBooleans(int rowId, int count, boolean value) { - byte v = (byte)((value) ? 1 : 0); - for (int i = 0; i < count; ++i) { - Platform.putByte(data, offset + i + rowId, v); - } - } - - @Override - public final boolean getBoolean(int rowId) { - return Platform.getByte(data, offset + rowId) == 1; - } - - // - - // - // APIs dealing with Bytes - // - - @Override - public final void putByte(int rowId, byte value) { - Platform.putByte(data, offset + rowId, value); - } - - @Override - public final void putBytes(int rowId, int count, byte value) { - for (int i = 0; i < count; ++i) { - Platform.putByte(data, offset + i + rowId, value); - } - } - - @Override - public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { - System.arraycopy(src, srcIndex, data, rowId, count); - } - - @Override - public final byte getByte(int rowId) { - if (dictionary == null) { - return Platform.getByte(data, offset + rowId); - } else { - return (byte) dictionary.decodeToInt(dictionaryIds.getInt(rowId)); - } - } - - // - // APIs dealing with Shorts - // - - @Override - public final void putShort(int rowId, short value) { - Platform.putShort(data, offset + rowId * 2, value); - } - - @Override - public final void putShorts(int rowId, int count, short value) { - for (int i = 0; i < count; ++i) { - Platform.putShort(data, offset + (i + rowId) * 2, value); - } - } - - @Override - public final void putShorts(int rowId, int count, short[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final short getShort(int rowId) { - if (dictionary == null) { - return Platform.getShort(data, offset + rowId * 2); - } else { - return (short) dictionary.decodeToInt(dictionaryIds.getInt(rowId)); - } - } - - - // - // APIs dealing with Ints - // - - @Override - public final void putInt(int rowId, int value) { - Platform.putInt(data, offset + rowId * 4, value); - } - - @Override - public final void putInts(int rowId, int count, int value) { - for (int i = 0; i < count; ++i) { - Platform.putInt(data, offset + (i + rowId) * 4, value); - } - } - - @Override - public final void putInts(int rowId, int count, int[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final int getInt(int rowId) { - if (dictionary == null) { - return Platform.getInt(data, offset + rowId * 4); - } else { - return dictionary.decodeToInt(dictionaryIds.getInt(rowId)); - } - } - - // - // APIs dealing with Longs - // - - @Override - public final void putLong(int rowId, long value) { - Platform.putLong(data, offset + rowId * 8, value); - } - - @Override - public final void putLongs(int rowId, int count, long value) { - for (int i = 0; i < count; ++i) { - Platform.getLong(data, offset + (i + rowId) * 8); - } - } - - @Override - public final void putLongs(int rowId, int count, long[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final long getLong(int rowId) { - if (dictionary == null) { - return Platform.getLong(data, offset + rowId * 8); - } else { - return dictionary.decodeToLong(dictionaryIds.getInt(rowId)); - } - } - - // - // APIs dealing with floats - // - - @Override - public final void putFloat(int rowId, float value) { Platform.putFloat(data, offset + rowId * 4, value); } - - @Override - public final void putFloats(int rowId, int count, float value) { - throw new NotImplementedException(); - } - - @Override - public final void putFloats(int rowId, int count, float[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); - } - - @Override - public final float getFloat(int rowId) { - if (dictionary == null) { - return Platform.getFloat(data, offset + rowId * 4); - } else { - return dictionary.decodeToFloat(dictionaryIds.getInt(rowId)); - } - } - - // - // APIs dealing with doubles - // - - @Override - public final void putDouble(int rowId, double value) { Platform.putDouble(data, offset + rowId * 8, value); } - - @Override - public final void putDoubles(int rowId, int count, double value) { - throw new NotImplementedException(); - } - - @Override - public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { - throw new NotImplementedException(); - } - - @Override - public final double getDouble(int rowId) { - if (dictionary == null) { - return Platform.getDouble(data, offset + rowId * 8); - } else { - return dictionary.decodeToDouble(dictionaryIds.getInt(rowId)); - } - } - - // - // APIs dealing with Arrays - // - - @Override - public final int getArrayLength(int rowId) { throw new NotImplementedException(); } - @Override - public final int getArrayOffset(int rowId) { throw new NotImplementedException(); } - - @Override - public final void putArray(int rowId, int offset, int length) { - throw new NotImplementedException(); - } - - @Override - public final void loadBytes(ColumnVector.Array array) { - throw new NotImplementedException(); - } - - // - // APIs dealing with Byte Arrays - // - - @Override - public final int putByteArray(int rowId, byte[] value, int offset, int length) { - throw new NotImplementedException(); - } - - @Override - public final void reserve(int requiredCapacity) { - if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); - } - - // Spilt this function out since it is the slow path. - private final void reserveInternal(int newCapacity) { - throw new NotImplementedException(); - } -} +/* + * 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 java.util.Arrays; + +import org.apache.commons.lang.NotImplementedException; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow; +import org.apache.spark.sql.catalyst.expressions.MutableRow; +import org.apache.spark.sql.execution.columnar.BasicColumnAccessor; +import org.apache.spark.sql.execution.columnar.ByteBufferHelper; +import org.apache.spark.sql.execution.columnar.NativeColumnAccessor; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.Platform; + +/** + * A column backed by an in memory JVM array. This stores the NULLs as a byte per value + * and a java array for the values. + */ +public final class ByteBufferColumnVector extends ColumnVector { + // The data stored in these arrays need to maintain binary compatible. We can + // directly pass this buffer to external components. + + // This is faster than a boolean array and we optimize this over memory footprint. + private byte[] nulls; + + // Array for each type. Only 1 is populated for any type. + private byte[] data; + private long offset; + + protected ByteBufferColumnVector(int capacity, DataType type, + boolean isConstant, ByteBuffer buffer, ByteBuffer nullsBuffer) { + super(capacity, type, MemoryMode.ON_HEAP); + if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { + throw new NotImplementedException(); + } else if (type instanceof BooleanType || type instanceof ByteType || + type instanceof ShortType || + type instanceof IntegerType || type instanceof DateType || + DecimalType.is32BitDecimalType(type) || + type instanceof LongType || DecimalType.is64BitDecimalType(type) || + (type instanceof FloatType) || (type instanceof DoubleType)) { + data = buffer.array(); + offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); + } else if (resultStruct != null) { + // Nothing to store. + } else { + throw new RuntimeException("Unhandled " + type); + } + nulls = new byte[capacity]; + reset(); + + int numNulls = ByteBufferHelper.getInt(nullsBuffer); + for (int i = 0; i < numNulls; i++) { + int cordinal = ByteBufferHelper.getInt(nullsBuffer); + putNull(cordinal); + } + if (isConstant) { + setIsConstant(); + } + } + + @Override + public final long valuesNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + @Override + public final long nullsNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + + @Override + public final void close() { + } + + // + // APIs dealing with nulls + // + + @Override + public final void putNotNull(int rowId) { + nulls[rowId] = (byte)0; + } + + @Override + public final void putNull(int rowId) { + nulls[rowId] = (byte)1; + ++numNulls; + anyNullsSet = true; + } + + @Override + public final void putNulls(int rowId, int count) { + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)1; + } + anyNullsSet = true; + numNulls += count; + } + + @Override + public final void putNotNulls(int rowId, int count) { + if (!anyNullsSet) return; + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)0; + } + } + + @Override + public final boolean isNullAt(int rowId) { + return nulls[rowId] == 1; + } + + // + // APIs dealing with Booleans + // + + @Override + public final void putBoolean(int rowId, boolean value) { + Platform.putByte(data, offset + rowId, (byte)((value) ? 1 : 0)); + } + + @Override + public final void putBooleans(int rowId, int count, boolean value) { + byte v = (byte)((value) ? 1 : 0); + for (int i = 0; i < count; ++i) { + Platform.putByte(data, offset + i + rowId, v); + } + } + + @Override + public final boolean getBoolean(int rowId) { + return Platform.getByte(data, offset + rowId) == 1; + } + + // + + // + // APIs dealing with Bytes + // + + @Override + public final void putByte(int rowId, byte value) { + Platform.putByte(data, offset + rowId, value); + } + + @Override + public final void putBytes(int rowId, int count, byte value) { + for (int i = 0; i < count; ++i) { + Platform.putByte(data, offset + i + rowId, value); + } + } + + @Override + public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { + System.arraycopy(src, srcIndex, data, rowId, count); + } + + @Override + public final byte getByte(int rowId) { + assert(dictionary == null); + return Platform.getByte(data, offset + rowId); + } + + // + // APIs dealing with Shorts + // + + @Override + public final void putShort(int rowId, short value) { + Platform.putShort(data, offset + rowId * 2, value); + } + + @Override + public final void putShorts(int rowId, int count, short value) { + for (int i = 0; i < count; ++i) { + Platform.putShort(data, offset + (i + rowId) * 2, value); + } + } + + @Override + public final void putShorts(int rowId, int count, short[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final short getShort(int rowId) { + assert(dictionary == null); + return Platform.getShort(data, offset + rowId * 2); + } + + + // + // APIs dealing with Ints + // + + @Override + public final void putInt(int rowId, int value) { + Platform.putInt(data, offset + rowId * 4, value); + } + + @Override + public final void putInts(int rowId, int count, int value) { + for (int i = 0; i < count; ++i) { + Platform.putInt(data, offset + (i + rowId) * 4, value); + } + } + + @Override + public final void putInts(int rowId, int count, int[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final int getInt(int rowId) { + assert(dictionary == null); + return Platform.getInt(data, offset + rowId * 4); + } + + // + // APIs dealing with Longs + // + + @Override + public final void putLong(int rowId, long value) { + Platform.putLong(data, offset + rowId * 8, value); + } + + @Override + public final void putLongs(int rowId, int count, long value) { + for (int i = 0; i < count; ++i) { + Platform.putLong(data, offset + (i + rowId) * 8, value); + } + } + + @Override + public final void putLongs(int rowId, int count, long[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final long getLong(int rowId) { + assert(dictionary == null); + return Platform.getLong(data, offset + rowId * 8); + } + + // + // APIs dealing with floats + // + + @Override + public final void putFloat(int rowId, float value) { Platform.putFloat(data, offset + rowId * 4, value); } + + @Override + public final void putFloats(int rowId, int count, float value) { + throw new NotImplementedException(); + } + + @Override + public final void putFloats(int rowId, int count, float[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + data, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + } + + @Override + public final float getFloat(int rowId) { + assert(dictionary == null); + return Platform.getFloat(data, offset + rowId * 4); + } + + // + // APIs dealing with doubles + // + + @Override + public final void putDouble(int rowId, double value) { Platform.putDouble(data, offset + rowId * 8, value); } + + @Override + public final void putDoubles(int rowId, int count, double value) { + throw new NotImplementedException(); + } + + @Override + public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + throw new NotImplementedException(); + } + + @Override + public final double getDouble(int rowId) { + assert(dictionary == null); + return Platform.getDouble(data, offset + rowId * 8); + } + + // + // APIs dealing with Arrays + // + + @Override + public final int getArrayLength(int rowId) { throw new NotImplementedException(); } + @Override + public final int getArrayOffset(int rowId) { throw new NotImplementedException(); } + + @Override + public final void putArray(int rowId, int offset, int length) { + throw new NotImplementedException(); + } + + @Override + public final void loadBytes(ColumnVector.Array array) { + throw new NotImplementedException(); + } + + // + // APIs dealing with Byte Arrays + // + + @Override + public final int putByteArray(int rowId, byte[] value, int offset, int length) { + throw new NotImplementedException(); + } + + @Override + public final void reserve(int requiredCapacity) { + if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); + } + + // Spilt this function out since it is the slow path. + private final void reserveInternal(int newCapacity) { + throw new NotImplementedException(); + } +} From 78a8258f0d4a000426041b5b4c5b08998a0caffb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 16 Apr 2016 15:19:36 -0400 Subject: [PATCH 24/50] Walk over an iterator when a loop is finished since an aggregation is once called this stuff This fixes failure of auto_join24 --- .../expressions/codegen/CodeGenerator.scala | 1 + .../sql/execution/WholeStageCodegenExec.scala | 54 ++++++++++--------- 2 files changed, 30 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 2bb35c4715bc..4e8c7065d144 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -213,6 +213,7 @@ class CodegenContext { var isRow = true var enableColumnCodeGen = false var inputHolder = "" + val columnarItrName = "columnar_iterator" val columnarBatchName = "columnar_batch" val columnarBatchIdxName = "columnar_batchIdx" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index f3e54ed23595..cd32d1c981ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -275,17 +275,28 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).gen(ctx) } s""" - | $columnarBatchClz batch = ($columnarBatchClz) $batch; + | while (true) { + | if (${idx} == 0) { + | if (${ctx.columnarItrName}.hasNext()) { + | ${batch} = ${ctx.columnarItrName}.next(); + | } else { + | cleanup(); + | break; + | } + | } | - | if ($idx == 0) { - | ${columnAssigns.mkString("", "\n", "")} - | } + | $columnarBatchClz batch = ($columnarBatchClz) $batch; + | if ($idx == 0) { + | ${columnAssigns.mkString("", "\n", "")} + | } | - | int $numrows = batch.numRows(); - | while ($idx < $numrows) { - | int $rowidx = $idx++; - | ${consume(ctx, columns, null).trim} - | if (shouldStop()) return; + | int $numrows = batch.numRows(); + | while ($idx < $numrows) { + | int $rowidx = $idx++; + | ${consume(ctx, columns, null).trim} + | if (shouldStop()) return; + | } + | $idx = 0; | } """.stripMargin } @@ -368,6 +379,8 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co child.asInstanceOf[CodegenSupport].produce(ctx, this) } else null + ctx.addMutableState("scala.collection.Iterator", ctx.columnarItrName, + s"${ctx.columnarItrName} = null;") ctx.addMutableState("Object", ctx.columnarBatchName, s"${ctx.columnarBatchName} = null;") ctx.addMutableState("int", ctx.columnarBatchIdxName, s"${ctx.columnarBatchIdxName} = 0;") @@ -379,20 +392,8 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """ } else { s""" - private void processBatch(scala.collection.Iterator itr) throws java.io.IOException { - while (true) { - if (${ctx.columnarBatchIdxName} == 0) { - if (itr.hasNext()) { - ${ctx.columnarBatchName} = itr.next(); - } else { - cleanup(); - return; - } - } - ${codeCol.trim} - - ${ctx.columnarBatchIdxName} = 0; - } + private void processBatch() throws java.io.IOException { + ${codeCol.trim} } private void processRow() throws java.io.IOException { @@ -401,6 +402,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co private void cleanup() { ${ctx.columnarBatchName} = null; + ${ctx.columnarItrName} = null; ${ctx.cleanupMutableStates()} } @@ -409,13 +411,15 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co if (${ctx.columnarBatchName} != null) { columnItr = (org.apache.spark.sql.execution.columnar.ColumnarIterator) ${ctx.inputHolder}; - processBatch(columnItr.getInput()); + ${ctx.columnarItrName} = columnItr.getInput(); + processBatch(); } else if (${ctx.inputHolder} instanceof org.apache.spark.sql.execution.columnar.ColumnarIterator && ((columnItr = (org.apache.spark.sql.execution.columnar.ColumnarIterator) ${ctx.inputHolder}).isSupportColumnarCodeGen())) { ${ctx.columnarBatchIdxName} = 0; - processBatch(columnItr.getInput()); + ${ctx.columnarItrName} = columnItr.getInput(); + processBatch(); } else { processRow(); } From 5f93490033c5581e5335a9f9fef6cb9147cf1857 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 17 Apr 2016 09:59:15 -0400 Subject: [PATCH 25/50] remove unused code --- .../columnar/GenerateColumnAccessor.scala | 7 ++----- .../columnar/InMemoryTableScanExec.scala | 16 ++++++---------- 2 files changed, 8 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 2ffc9934ae84..82db169e2ad7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types._ */ abstract class ColumnarIterator extends Iterator[InternalRow] { def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], - columnIndexes: Array[Int], columnNullables: Array[Boolean]): Unit + columnIndexes: Array[Int]): Unit def getInput: Iterator[CachedBatch] def getColumnIndexes: Array[Int] def isSupportColumnarCodeGen: Boolean @@ -184,7 +184,6 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera private scala.collection.Iterator input = null; private DataType[] columnTypes = null; private int[] columnIndexes = null; - private boolean[] columnNullables = null; ${ctx.declareMutableStates()} @@ -194,12 +193,10 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera this.mutableRow = new MutableUnsafeRow(rowWriter); } - public void initialize(Iterator input, DataType[] columnTypes, int[] columnIndexes, - boolean columnNullables[]) { + public void initialize(Iterator input, DataType[] columnTypes, int[] columnIndexes) { this.input = input; this.columnTypes = columnTypes; this.columnIndexes = columnIndexes; - this.columnNullables = columnNullables; } ${ctx.declareAddedFunctions()} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 576cd61734bd..fc0d05cda252 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -133,9 +133,9 @@ private[sql] case class InMemoryTableScanExec( schema) // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnAttribute) = + val (requestedColumnIndices, requestedColumnDataTypes) = attributes.map { a => - relOutput.indexOf(a.exprId) -> a + relOutput.indexOf(a.exprId) -> a.dataType }.unzip // Do partition batch pruning if enabled @@ -167,16 +167,12 @@ private[sql] case class InMemoryTableScanExec( batch } - val columnTypes = requestedColumnAttribute.map { a => - a.dataType match { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - } + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other }.toArray - val columnNullables = requestedColumnAttribute.map { _.nullable }.toArray val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray, - columnNullables) + columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) if (enableAccumulators && columnarIterator.hasNext) { readPartitions.add(1) } From a8708ab4ca40af6b1a4a1ae48dd8c0d1531e32e8 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 17 Apr 2016 10:00:45 -0400 Subject: [PATCH 26/50] Duplicate ByteBuffer in a column for CachedBatch --- .../apache/spark/sql/execution/columnar/ColumnAccessor.scala | 3 ++- .../execution/columnar/compression/compressionSchemes.scala | 3 ++- 2 files changed, 4 insertions(+), 2 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 1b41cb380624..08c2b3913c2e 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,7 +62,8 @@ private[columnar] abstract class BasicColumnAccessor[JvmType]( protected def underlyingBuffer = buffer - def getByteBuffer: ByteBuffer = buffer + def getByteBuffer: ByteBuffer = + buffer.duplicate.order(ByteOrder.nativeOrder()) } private[columnar] class NullColumnAccessor(buffer: ByteBuffer) 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 84bf839bda56..3d3676c58415 100755 --- 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 @@ -69,7 +69,7 @@ private[columnar] case object PassThrough extends CompressionScheme { val nullCount = ByteBufferHelper.getInt(nullsBuffer) if (nullCount == 0) { nullsBuffer.rewind() - (buffer, nullsBuffer) + (buffer.duplicate().order(ByteOrder.nativeOrder()), nullsBuffer) } else { val unitSize = columnType.dataType match { case _: BooleanType => 1 @@ -210,6 +210,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { } else { valueCount += 1 } + columnType.setField(row, ordinal, currentValue) } From d456df7c41e370885e6aefbe17b87b6db7d3609b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 17 Apr 2016 10:08:24 -0400 Subject: [PATCH 27/50] Move string constants to WholeStageCodegen --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 4e8c7065d144..e56e2922074e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -212,10 +212,7 @@ class CodegenContext { var isRow = true var enableColumnCodeGen = false - var inputHolder = "" - val columnarItrName = "columnar_iterator" - val columnarBatchName = "columnar_batch" - val columnarBatchIdxName = "columnar_batchIdx" + var iteratorInput = "" /** * The map from a variable name to it's next ID. From 257227fee213d0963364b1f68a708fcd3c71ee32 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 17 Apr 2016 10:10:11 -0400 Subject: [PATCH 28/50] avoid generating code for CachedBatch since these routine generates code to directly access InternalRow --- .../main/scala/org/apache/spark/sql/execution/SortExec.scala | 2 +- .../spark/sql/execution/joins/BroadcastHashJoinExec.scala | 2 ++ .../apache/spark/sql/execution/joins/SortMergeJoinExec.scala | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index f9aac8bcc587..b414b03724e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -122,7 +122,7 @@ case class SortExec( // Name of sorter variable used in codegen. private var sorterVariable: String = _ - override def preferUnsafeRow: Boolean = true + override def useUnsafeRow: Boolean = true override protected def doProduce(ctx: CodegenContext): String = { val needToSort = ctx.freshName("needToSort") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 7c194ab72643..93ec860c95ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -73,6 +73,8 @@ case class BroadcastHashJoinExec( streamedPlan.asInstanceOf[CodegenSupport].inputRDDs() } + override def useUnsafeRow: Boolean = true + override def doProduce(ctx: CodegenContext): String = { streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index fac6b8de8ed5..38e6f681b9d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -502,6 +502,8 @@ case class SortMergeJoinExec( } } + override def useUnsafeRow: Boolean = true + override def doProduce(ctx: CodegenContext): String = { ctx.copyResult = true val leftInput = ctx.freshName("leftInput") From 598b9ff7e159084d43e8d792a491be3d9d7d2ac7 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 17 Apr 2016 10:10:50 -0400 Subject: [PATCH 29/50] Make code simple --- .../sql/execution/WholeStageCodegenExec.scala | 89 +++++++++---------- .../execution/basicPhysicalOperators.scala | 2 +- 2 files changed, 41 insertions(+), 50 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index cd32d1c981ea..3d0442821aec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -69,9 +69,9 @@ trait CodegenSupport extends SparkPlan { protected var parent: CodegenSupport = null /** - * Whether this SparkPlan prefers to accept UnsafeRow as input in doConsume + * Whether this SparkPlan uses UnsafeRow as input in doProduce or doConsume */ - def preferUnsafeRow: Boolean = false + def useUnsafeRow: Boolean = false /** * Returns all the RDDs of InternalRow which generates the input rows. @@ -244,7 +244,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp override def doProduce(ctx: CodegenContext): String = { ctx.enableColumnCodeGen = true val input = ctx.freshName("input") - ctx.inputHolder = input + ctx.iteratorInput = input // Right now, InputAdapter is only used when there is one input RDD. ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") @@ -261,31 +261,33 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp val columnarBatchClz = "org.apache.spark.sql.execution.columnar.CachedBatch" val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" - val batch = ctx.columnarBatchName - val idx = ctx.columnarBatchIdxName + val batch = WholeStageCodegen.columnarBatchName + val itr = WholeStageCodegen.columnarItrName + val idx = WholeStageCodegen.columnarBatchIdxName + ctx.addMutableState("scala.collection.Iterator", itr, s"$itr = null;", s"$itr = null;") + ctx.addMutableState("Object", batch, s"$batch = null;", s"$batch = null;") + ctx.addMutableState("int", idx, s"$idx = 0;") val rowidx = ctx.freshName("rowIdx") - val col = ctx.freshName("col") val numrows = ctx.freshName("numRows") val colVars = output.indices.map(i => ctx.freshName("col" + i)) val columnAssigns = colVars.zipWithIndex.map { case (name, i) => ctx.addMutableState(columnVectorClz, name, s"$name = null;", s"$name = null;") s"$name = batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } - ctx.currentVars = null val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).gen(ctx) } s""" | while (true) { - | if (${idx} == 0) { - | if (${ctx.columnarItrName}.hasNext()) { - | ${batch} = ${ctx.columnarItrName}.next(); + | if ($idx == 0) { + | if ($itr.hasNext()) { + | $batch = $itr.next(); | } else { | cleanup(); | break; | } | } | - | $columnarBatchClz batch = ($columnarBatchClz) $batch; + | $columnarBatchClz batch = ($columnarBatchClz)$batch; | if ($idx == 0) { | ${columnAssigns.mkString("", "\n", "")} | } @@ -314,6 +316,9 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp object WholeStageCodegenExec { val PIPELINE_DURATION_METRIC = "duration" + val columnarItrName = "columnar_iterator" + val columnarBatchName = "columnar_batch" + val columnarBatchIdxName = "columnar_batchIdx" } /** @@ -363,34 +368,25 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co val ctx = new CodegenContext ctx.isRow = true val codeRow = child.asInstanceOf[CodegenSupport].produce(ctx, this) - val referUnsafeRow = child.find(c => c.isInstanceOf[CodegenSupport] && - c.asInstanceOf[CodegenSupport].preferUnsafeRow) match { - case Some(c) => true - case None => false - } - val useInMemoryColumnar = child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]) match { - case Some(c) => true - case None => false - } - val enableColumnCodeGen = ctx.enableColumnCodeGen && !referUnsafeRow && useInMemoryColumnar && - sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean - val codeCol = if (enableColumnCodeGen) { - ctx.isRow = false - child.asInstanceOf[CodegenSupport].produce(ctx, this) - } else null - ctx.addMutableState("scala.collection.Iterator", ctx.columnarItrName, - s"${ctx.columnarItrName} = null;") - ctx.addMutableState("Object", ctx.columnarBatchName, s"${ctx.columnarBatchName} = null;") - ctx.addMutableState("int", ctx.columnarBatchIdxName, s"${ctx.columnarBatchIdxName} = 0;") + val useUnsafeRow = child.find(c => c.isInstanceOf[CodegenSupport] && + c.asInstanceOf[CodegenSupport].useUnsafeRow + ).isDefined + val useInMemoryColumnar = child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]).isDefined + val enableColumnCodeGen = ctx.enableColumnCodeGen && !useUnsafeRow && useInMemoryColumnar && + sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean - val codeProcess = if (codeCol == null) { + val codeProcessNext = if (!enableColumnCodeGen) { s""" - protected void processNext() throws java.io.IOException { - ${codeRow.trim} - } - """ + protected void processNext() throws java.io.IOException { + ${codeRow.trim} + } + """ } else { + ctx.isRow = false + val codeCol = child.asInstanceOf[CodegenSupport].produce(ctx, this) + + val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" s""" private void processBatch() throws java.io.IOException { ${codeCol.trim} @@ -401,24 +397,19 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } private void cleanup() { - ${ctx.columnarBatchName} = null; - ${ctx.columnarItrName} = null; ${ctx.cleanupMutableStates()} } protected void processNext() throws java.io.IOException { - org.apache.spark.sql.execution.columnar.ColumnarIterator columnItr = null; - if (${ctx.columnarBatchName} != null) { - columnItr = (org.apache.spark.sql.execution.columnar.ColumnarIterator) - ${ctx.inputHolder}; - ${ctx.columnarItrName} = columnItr.getInput(); + $columnarItrClz columnItr = null; + if (${WholeStageCodegen.columnarBatchName} != null) { + columnItr = ($columnarItrClz)${ctx.iteratorInput}; + ${WholeStageCodegen.columnarItrName} = columnItr.getInput(); processBatch(); - } else if (${ctx.inputHolder} instanceof - org.apache.spark.sql.execution.columnar.ColumnarIterator && - ((columnItr = (org.apache.spark.sql.execution.columnar.ColumnarIterator) - ${ctx.inputHolder}).isSupportColumnarCodeGen())) { - ${ctx.columnarBatchIdxName} = 0; - ${ctx.columnarItrName} = columnItr.getInput(); + } else if (${ctx.iteratorInput} instanceof $columnarItrClz && + ((columnItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { + ${WholeStageCodegen.columnarBatchIdxName} = 0; + ${WholeStageCodegen.columnarItrName} = columnItr.getInput(); processBatch(); } else { processRow(); @@ -449,7 +440,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co ${ctx.declareAddedFunctions()} - ${codeProcess} + ${codeProcessNext} } """.trim diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index dfad3c2066e5..5ed78aa4a996 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -382,7 +382,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) """.stripMargin) val input = ctx.freshName("input") - ctx.inputHolder = input + ctx.iteratorInput = input // Right now, Range is only used when there is one upstream. ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") s""" From 9a79a57d5646c6671ce0efb5f954f927bac5c14b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 17 Apr 2016 10:15:19 -0400 Subject: [PATCH 30/50] Add benchmark suites with results --- .../spark/sql/DataFrameCacheBenchmark.scala | 262 ++++++++++++++++++ 1 file changed, 262 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala new file mode 100644 index 000000000000..fd1ee8618e8d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala @@ -0,0 +1,262 @@ +/* + * 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 + +import java.util.Random + +import scala.util.Try + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure performance of columnar storage for dataframe cache. + * To run this: + * spark-submit --class + */ +object DataFrameCacheBenchmark { + val conf = new SparkConf() + val sc = new SparkContext("local[1]", "test-sql-context", conf) + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Set default configs. Individual cases will change them if necessary. + sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + + def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(sqlContext.conf.getConfString(key)).toOption) + (keys, values).zipped.foreach(sqlContext.conf.setConfString) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => sqlContext.conf.setConfString(key, value) + case (key, None) => sqlContext.conf.unsetConf(key) + } + } + } + + def intSumBenchmark(values: Int, iters: Int = 5): Unit = { + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) + + val benchmarkPT = new Benchmark("Int Sum with PassThrough cache", values, iters) + val rand1 = new Random(511) + val dfPassThrough = sc.parallelize(0 to values - 1, 1).map(i => rand1.nextInt()).toDF.cache() + dfPassThrough.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkPT.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfPassThrough.agg(sum("value")).collect + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Int Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 462 / 466 68.1 14.7 1.0X + ColumnVector codegen 94 / 100 336.3 3.0 4.9X + */ + + benchmarkPT.run() + dfPassThrough.unpersist(true) + System.gc() + + val benchmarkRL = new Benchmark("Int Sum with RunLength cache", values, iters) + val dfRunLength = sc.parallelize(0 to values - 1, 1) + .map(i => (i, (i / 1024).toInt)).toDF("k", "v").cache() + dfRunLength.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkRL.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfRunLength.agg(sum("v")).collect() + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Int Sum with RunLength cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 492 / 553 63.9 15.7 1.0X + ColumnVector codegen 175 / 180 179.5 5.6 2.8X + */ + + benchmarkRL.run() + dfRunLength.unpersist(true) + System.gc() + + val rand2 = new Random(42) + val benchmarkDIC = new Benchmark("Int Sum with Dictionary cache", values, iters) + val dfDictionary = sc.parallelize(0 to values - 1, 1) + .map(i => (i, (rand2.nextInt() % 1023))).toDF("k", "v").cache() + dfDictionary.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkDIC.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfDictionary.agg(sum("v")).collect() + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Int Sum with Dictionary cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 599 / 610 52.5 19.0 1.0X + ColumnVector codegen 320 / 327 98.3 10.2 1.9X + */ + + benchmarkDIC.run() + dfDictionary.unpersist(true) + System.gc() + + val benchmarkIDelta = new Benchmark("Int Sum with Delta cache", values, iters) + val dfIntDelta = sc.parallelize(0 to values - 1, 1) + .map(i => (i, i)).toDF("k", "v").cache() + dfIntDelta.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkIDelta.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfIntDelta.agg(sum("v")).collect() + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Int Sum with Delta cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 467 / 512 67.4 14.8 1.0X + ColumnVector codegen 247 / 250 127.4 7.8 1.9X + */ + + benchmarkIDelta.run() + dfIntDelta.unpersist(true) + System.gc() + } + + def longSumBenchmark(values: Int, iters: Int = 5): Unit = { + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) + + val benchmarkPT = new Benchmark("Long Sum with PassThrough cache", values, iters) + val rand1 = new Random(511) + val dfPassThrough = sc.parallelize(0 to values - 1, 1).map(i => rand1.nextLong()).toDF().cache() + dfPassThrough.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkPT.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfPassThrough.agg(sum("value")).collect + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Long Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 382 / 420 41.2 24.3 1.0X + ColumnVector codegen 89 / 101 176.2 5.7 4.3X + */ + + benchmarkPT.run() + dfPassThrough.unpersist(true) + System.gc() + } + + def floatSumBenchmark(values: Int, iters: Int = 5): Unit = { + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) + + val benchmarkPT = new Benchmark("Float Sum with PassThrough cache", values, iters) + val rand1 = new Random(511) + val dfPassThrough = sc.parallelize(0 to values - 1, 1) + .map(i => rand1.nextFloat()).toDF().cache() + dfPassThrough.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkPT.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfPassThrough.agg(sum("value")).collect + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Float Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 476 / 483 66.1 15.1 1.0X + ColumnVector codegen 91 / 103 343.8 2.9 5.2X + */ + + benchmarkPT.run() + dfPassThrough.unpersist(true) + System.gc() + } + + def doubleSumBenchmark(values: Int, iters: Int = 5): Unit = { + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) + + val benchmarkPT = new Benchmark("Double Sum with PassThrough cache", values, iters) + val rand1 = new Random(511) + val dfPassThrough = sc.parallelize(0 to values - 1, 1) + .map(i => rand1.nextDouble()).toDF().cache() + dfPassThrough.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkPT.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfPassThrough.agg(sum("value")).collect + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Double Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 290 / 306 54.3 18.4 1.0X + ColumnVector codegen 95 / 101 165.7 6.0 3.1X + */ + + benchmarkPT.run() + dfPassThrough.unpersist(true) + System.gc() + } + + def main(args: Array[String]): Unit = { + longSumBenchmark(1024 * 1024 * 15) + doubleSumBenchmark(1024 * 1024 * 15) + floatSumBenchmark(1024 * 1024 * 30) + intSumBenchmark(1024 * 1024 * 30) + } +} From 9943698c557786031292ce9a2fcd4bf4795929a1 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 18 Apr 2016 02:17:54 +0900 Subject: [PATCH 31/50] Reduce a conditinal branch and cast in generated Java code --- .../sql/execution/WholeStageCodegenExec.scala | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 3d0442821aec..0d7a6e791270 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -264,15 +264,15 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp val batch = WholeStageCodegen.columnarBatchName val itr = WholeStageCodegen.columnarItrName val idx = WholeStageCodegen.columnarBatchIdxName + ctx.addMutableState(s"$columnarBatchClz", batch, s"$batch = null;", s"$batch = null;") ctx.addMutableState("scala.collection.Iterator", itr, s"$itr = null;", s"$itr = null;") - ctx.addMutableState("Object", batch, s"$batch = null;", s"$batch = null;") ctx.addMutableState("int", idx, s"$idx = 0;") val rowidx = ctx.freshName("rowIdx") val numrows = ctx.freshName("numRows") val colVars = output.indices.map(i => ctx.freshName("col" + i)) val columnAssigns = colVars.zipWithIndex.map { case (name, i) => ctx.addMutableState(columnVectorClz, name, s"$name = null;", s"$name = null;") - s"$name = batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } + s"$name = $batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).gen(ctx) } @@ -280,19 +280,15 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | while (true) { | if ($idx == 0) { | if ($itr.hasNext()) { - | $batch = $itr.next(); + | $batch = ($columnarBatchClz)($itr.next()); | } else { | cleanup(); | break; | } - | } - | - | $columnarBatchClz batch = ($columnarBatchClz)$batch; - | if ($idx == 0) { | ${columnAssigns.mkString("", "\n", "")} | } | - | int $numrows = batch.numRows(); + | int $numrows = $batch.numRows(); | while ($idx < $numrows) { | int $rowidx = $idx++; | ${consume(ctx, columns, null).trim} @@ -385,6 +381,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } else { ctx.isRow = false val codeCol = child.asInstanceOf[CodegenSupport].produce(ctx, this) + val columnItr = ctx.freshName("columnItr") val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" s""" @@ -401,15 +398,15 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } protected void processNext() throws java.io.IOException { - $columnarItrClz columnItr = null; + $columnarItrClz $columnItr = null; if (${WholeStageCodegen.columnarBatchName} != null) { - columnItr = ($columnarItrClz)${ctx.iteratorInput}; - ${WholeStageCodegen.columnarItrName} = columnItr.getInput(); + $columnItr = ($columnarItrClz)${ctx.iteratorInput}; + ${WholeStageCodegen.columnarItrName} = $columnItr.getInput(); processBatch(); } else if (${ctx.iteratorInput} instanceof $columnarItrClz && - ((columnItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { + (($columnItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { ${WholeStageCodegen.columnarBatchIdxName} = 0; - ${WholeStageCodegen.columnarItrName} = columnItr.getInput(); + ${WholeStageCodegen.columnarItrName} = $columnItr.getInput(); processBatch(); } else { processRow(); From 1b7e26f52538b068d81aefc5ac77886b091f694a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Apr 2016 03:18:30 -0400 Subject: [PATCH 32/50] rebase --- .../sql/catalyst/expressions/BoundAttribute.scala | 12 ++++-------- .../spark/sql/execution/WholeStageCodegenExec.scala | 2 +- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index a381c09230f3..00d4a26c846a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -113,19 +113,15 @@ case class ColumnVectorReference( override def eval(input: InternalRow): Any = null - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) val value = ctx.getValue(columnVar, dataType, ordinal) if (nullable) { - s""" + ev.copy(code = s""" boolean ${ev.isNull} = ${columnVar}.isNullAt($ordinal); - $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : ($value); - """ + $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : ($value);""") } else { - ev.isNull = "false" - s""" - $javaType ${ev.value} = $value; - """ + ev.copy(code = s"""$javaType ${ev.value} = $value;""", isNull = "false") } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 0d7a6e791270..d7b54d30cebe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -275,7 +275,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp s"$name = $batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } val columns = (output zip colVars).map { case (attr, colVar) => - new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).gen(ctx) } + new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } s""" | while (true) { | if ($idx == 0) { From 7426f6fff69384f58eacbd8e5b81cf59c79ec2fb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Apr 2016 11:40:11 -0400 Subject: [PATCH 33/50] add new test suites for decompress() and codegen for Dataframe.cache() --- .../sql/execution/WholeStageCodegenExec.scala | 9 +- .../sql/execution/columnar/ColumnType.scala | 6 + .../sql/execution/DataFrameCacheSuite.scala | 103 +++++++++ .../compression/BooleanBitSetSuite.scala | 51 +++++ .../compression/DictionaryEncodingSuite.scala | 80 ++++++- .../compression/IntegralDeltaSuite.scala | 78 +++++++ .../compression/PassThroughSuite.scala | 196 ++++++++++++++++++ .../compression/RunLengthEncodingSuite.scala | 96 ++++++++- 8 files changed, 611 insertions(+), 8 deletions(-) create mode 100755 sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala create mode 100755 sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index d7b54d30cebe..4fe8fe019903 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -279,12 +279,11 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp s""" | while (true) { | if ($idx == 0) { - | if ($itr.hasNext()) { - | $batch = ($columnarBatchClz)($itr.next()); - | } else { + | if (!$itr.hasNext()) { | cleanup(); | break; | } + | $batch = ($columnarBatchClz)($itr.next()); | ${columnAssigns.mkString("", "\n", "")} | } | @@ -355,6 +354,8 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext, WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) + var enableColumnCodeGen: Boolean = false + /** * Generates code for this subtree. * @@ -369,7 +370,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co c.asInstanceOf[CodegenSupport].useUnsafeRow ).isDefined val useInMemoryColumnar = child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]).isDefined - val enableColumnCodeGen = ctx.enableColumnCodeGen && !useUnsafeRow && useInMemoryColumnar && + enableColumnCodeGen = ctx.enableColumnCodeGen && !useUnsafeRow && useInMemoryColumnar && sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean val codeProcessNext = if (!enableColumnCodeGen) { 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 718dc6c5d406..23ec157ac531 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) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala new file mode 100755 index 000000000000..123a0bde1205 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala @@ -0,0 +1,103 @@ +/* + * 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 + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class DataFrameCacheSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("range/filter should be combined with column codegen") { + val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + .filter("value = 1").selectExpr("value + 1") + assert(df.collect() === Array(Row(2))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + } + + test("filters should be combined with column codegen") { + val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + .filter("value % 2 == 0").filter("value % 3 == 0") + assert(df.collect() === Array(Row(0), Row(6))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + } + + test("filter with null should be included in WholeStageCodegen with column codegen") { + val toInt = udf[java.lang.Integer, String] { s => if (s == "2") null else s.toInt } + val df0 = sparkContext.parallelize(0 to 4, 1).map(i => i.toString).toDF() + val df = df0.withColumn("i", toInt(df0("value"))).select("i").toDF().cache() + .filter("i % 2 == 0") + assert(df.collect() === Array(Row(0), Row(4))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + } + + test("Aggregate should be included in WholeStageCodegen with column codegen") { + val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + .groupBy().agg(max(col("value")), avg(col("value"))) + assert(df.collect() === Array(Row(9, 4.5))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + } + + test("Aggregate with grouping keys should be included in WholeStageCodegen with column codegen") { + val df = sparkContext.parallelize(0 to 2, 1).toDF().cache() + .groupBy("value").count().orderBy("value") + assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + } + + test("Aggregate with columns should be included in WholeStageCodegen with column codegen") { + val df = sparkContext.parallelize(0 to 10, 1).map(i => (i, (i * 2).toDouble)).toDF("i", "d") + .cache().agg(sum("d")) + assert(df.collect() === Array(Row(110.0))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + } + + test("Sort should be included in WholeStageCodegen without column codegen") { + val df = sqlContext.range(3, 0, -1).toDF().sort(col("id")) + val plan = df.queryExecution.executedPlan + assert(df.collect() === Array(Row(1), Row(2), Row(3))) + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + !p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[Sort]).isDefined) + } +} \ No newline at end of file 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 f67e9c7dae27..08390d6cdb15 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,37 @@ 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 +135,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 830ca0294e1b..84a138f8586e 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,15 @@ 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]) { + columnType: NativeColumnType[T], + testDecompress: Boolean = true) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") @@ -113,6 +115,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 GenericMutableRow(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 +186,19 @@ 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 988a577a7b4d..405f2e0463da 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,6 +24,7 @@ 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) @@ -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 GenericMutableRow(1) + val nullRow = new GenericMutableRow(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,27 @@ 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/PassThroughSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala new file mode 100755 index 000000000000..c9a9da9b132b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.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.GenericMutableRow +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 GenericMutableRow(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 GenericMutableRow(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 GenericMutableRow(1) + val nullRow = new GenericMutableRow(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])) + } + } +} 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 95642e93ae9f..86ef34f3cd83 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]) { + 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 GenericMutableRow(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)) + } } } From 8b192b57aa373dc30fcec4f444e6246d08840aa2 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Apr 2016 11:40:45 -0400 Subject: [PATCH 34/50] bug fixes --- .../columnar/compression/compressionSchemes.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 3d3676c58415..556a36512f50 100755 --- 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 @@ -269,7 +269,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { if (seenNulls < nullCount) { nextNullIndex = ByteBufferHelper.getInt(nullsBuffer) } - out.position(out.position + 2) + out.position(out.position + 1) } pos += 1 } @@ -603,7 +603,7 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { override def hasNext: Boolean = visited < count override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { - val countLocal = ByteBufferHelper.getInt(buffer) + val countLocal = count var currentWordLocal: Long = 0 var visitedLocal: Int = 0 val out = ByteBuffer.allocate(capacity).order(ByteOrder.nativeOrder()) @@ -623,7 +623,7 @@ private[columnar] case object BooleanBitSet extends CompressionScheme { currentWordLocal = ByteBufferHelper.getLong(buffer) } - out.put(if (((currentWord >> bit) & 1) != 0) 1: Byte else 0: Byte) + out.put(if (((currentWordLocal >> bit) & 1) != 0) 1: Byte else 0: Byte) } else { seenNulls += 1 if (seenNulls < nullCount) { @@ -831,7 +831,7 @@ private[columnar] case object LongDelta extends CompressionScheme { override def decompress(capacity: Int): (ByteBuffer, ByteBuffer) = { var prevLocal: Long = 0 - val out = ByteBuffer.allocate(capacity * 4).order(ByteOrder.nativeOrder()) + val out = ByteBuffer.allocate(capacity * 8).order(ByteOrder.nativeOrder()) val nullsBuffer = buffer.duplicate().order(ByteOrder.nativeOrder()) nullsBuffer.rewind val nullCount = ByteBufferHelper.getInt(nullsBuffer) From f48c947b953098d4c693621005947027ef696535 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Apr 2016 12:36:39 -0400 Subject: [PATCH 35/50] support accumulators in InMemoryColumnarTableScan --- .../sql/execution/WholeStageCodegenExec.scala | 12 +++++++++--- .../columnar/GenerateColumnAccessor.scala | 17 +++++++++++++++-- .../columnar/InMemoryTableScanExec.scala | 12 +++++++++--- 3 files changed, 33 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 4fe8fe019903..f01d666ee131 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -276,6 +276,13 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } + val enableAccumulators: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + val incrementReadPartitionAccumulator = if (enableAccumulators) { + s""" + (($columnarItrClz)$input).incrementReadPartitionAccumulator(); + """.trim + } else "" s""" | while (true) { | if ($idx == 0) { @@ -283,6 +290,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | cleanup(); | break; | } + | $incrementReadPartitionAccumulator | $batch = ($columnarBatchClz)($itr.next()); | ${columnAssigns.mkString("", "\n", "")} | } @@ -400,9 +408,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co protected void processNext() throws java.io.IOException { $columnarItrClz $columnItr = null; - if (${WholeStageCodegen.columnarBatchName} != null) { - $columnItr = ($columnarItrClz)${ctx.iteratorInput}; - ${WholeStageCodegen.columnarItrName} = $columnItr.getInput(); + if (${WholeStageCodegen.columnarItrName} != null) { processBatch(); } else if (${ctx.iteratorInput} instanceof $columnarItrClz && (($columnItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 82db169e2ad7..ae03a7925337 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,10 +28,11 @@ import org.apache.spark.sql.types._ */ abstract class ColumnarIterator extends Iterator[InternalRow] { def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], - columnIndexes: Array[Int]): Unit + columnIndexes: Array[Int], inMemoryColumnarTableScan: InMemoryColumnarTableScan): Unit def getInput: Iterator[CachedBatch] def getColumnIndexes: Array[Int] def isSupportColumnarCodeGen: Boolean + def incrementReadPartitionAccumulator: Unit } /** @@ -163,6 +164,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera import org.apache.spark.sql.types.DataType; 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.InMemoryColumnarTableScan; import org.apache.spark.sql.execution.columnar.MutableUnsafeRow; public SpecificColumnarIterator generate(Object[] references) { @@ -176,7 +178,9 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera private UnsafeRow unsafeRow = new UnsafeRow($numFields); private BufferHolder bufferHolder = new BufferHolder(unsafeRow); private UnsafeRowWriter rowWriter = new UnsafeRowWriter(bufferHolder, $numFields); + private InMemoryColumnarTableScan inMemoryColumnarTableScan = null; private MutableUnsafeRow mutableRow = null; + private boolean readPartitionIncremented = false; private int currentRow = 0; private int numRowsInBatch = 0; @@ -193,10 +197,12 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera this.mutableRow = new MutableUnsafeRow(rowWriter); } - public void initialize(Iterator input, DataType[] columnTypes, int[] columnIndexes) { + public void initialize(Iterator input, DataType[] columnTypes, + int[] columnIndexes, InMemoryColumnarTableScan inMemoryColumnarTableScan) { this.input = input; this.columnTypes = columnTypes; this.columnIndexes = columnIndexes; + this.inMemoryColumnarTableScan = inMemoryColumnarTableScan; } ${ctx.declareAddedFunctions()} @@ -236,6 +242,13 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera public boolean isSupportColumnarCodeGen() { return ${_isSupportColumnarCodeGen}; } + + public void incrementReadPartitionAccumulator() { + if (!readPartitionIncremented) { + inMemoryColumnarTableScan.incrementReadPartitionAccumulator(); + readPartitionIncremented = true; + } + } }""" val code = CodeFormatter.stripOverlappingComments( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index fc0d05cda252..d225d324df0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -110,6 +110,10 @@ private[sql] case class InMemoryTableScanExec( lazy val readPartitions = sparkContext.longAccumulator lazy val readBatches = sparkContext.longAccumulator + def incrementReadPartitionAccumulator(): Unit = { + readPartitions += 1 + } + private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning protected override def doExecute(): RDD[InternalRow] = { @@ -172,9 +176,11 @@ private[sql] case class InMemoryTableScanExec( case other => other }.toArray val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulators && columnarIterator.hasNext) { - readPartitions.add(1) + columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray, + if (!enableAccumulators) null else this) + if (enableAccumulators && !columnarIterator.isSupportColumnarCodeGen && + columnarIterator.hasNext) { + incrementReadPartitionAccumulator } columnarIterator } From 5f773c57f403bff39b5046a26d73f56a11a734cc Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Apr 2016 12:44:06 -0400 Subject: [PATCH 36/50] fix scalastyle and eol code --- .../sql/execution/DataFrameCacheSuite.scala | 206 +++++++++--------- 1 file changed, 103 insertions(+), 103 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala index 123a0bde1205..3fb7d5f4d195 100755 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala @@ -1,103 +1,103 @@ -/* - * 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 - -import org.apache.spark.sql.{QueryTest, Row} -import org.apache.spark.sql.execution.aggregate.TungstenAggregate -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext - -class DataFrameCacheSuite extends QueryTest with SharedSQLContext { - import testImplicits._ - - test("range/filter should be combined with column codegen") { - val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() - .filter("value = 1").selectExpr("value + 1") - assert(df.collect() === Array(Row(2))) - val plan = df.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) - } - - test("filters should be combined with column codegen") { - val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() - .filter("value % 2 == 0").filter("value % 3 == 0") - assert(df.collect() === Array(Row(0), Row(6))) - val plan = df.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) - } - - test("filter with null should be included in WholeStageCodegen with column codegen") { - val toInt = udf[java.lang.Integer, String] { s => if (s == "2") null else s.toInt } - val df0 = sparkContext.parallelize(0 to 4, 1).map(i => i.toString).toDF() - val df = df0.withColumn("i", toInt(df0("value"))).select("i").toDF().cache() - .filter("i % 2 == 0") - assert(df.collect() === Array(Row(0), Row(4))) - val plan = df.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) - } - - test("Aggregate should be included in WholeStageCodegen with column codegen") { - val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() - .groupBy().agg(max(col("value")), avg(col("value"))) - assert(df.collect() === Array(Row(9, 4.5))) - val plan = df.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) - } - - test("Aggregate with grouping keys should be included in WholeStageCodegen with column codegen") { - val df = sparkContext.parallelize(0 to 2, 1).toDF().cache() - .groupBy("value").count().orderBy("value") - assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) - val plan = df.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) - } - - test("Aggregate with columns should be included in WholeStageCodegen with column codegen") { - val df = sparkContext.parallelize(0 to 10, 1).map(i => (i, (i * 2).toDouble)).toDF("i", "d") - .cache().agg(sum("d")) - assert(df.collect() === Array(Row(110.0))) - val plan = df.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) - } - - test("Sort should be included in WholeStageCodegen without column codegen") { - val df = sqlContext.range(3, 0, -1).toDF().sort(col("id")) - val plan = df.queryExecution.executedPlan - assert(df.collect() === Array(Row(1), Row(2), Row(3))) - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - !p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[Sort]).isDefined) - } -} \ No newline at end of file +/* + * 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 + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class DataFrameCacheSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("range/filter should be combined with column codegen") { + val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + .filter("value = 1").selectExpr("value + 1") + assert(df.collect() === Array(Row(2))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + } + + test("filters should be combined with column codegen") { + val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + .filter("value % 2 == 0").filter("value % 3 == 0") + assert(df.collect() === Array(Row(0), Row(6))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + } + + test("filter with null should be included in WholeStageCodegen with column codegen") { + val toInt = udf[java.lang.Integer, String] { s => if (s == "2") null else s.toInt } + val df0 = sparkContext.parallelize(0 to 4, 1).map(i => i.toString).toDF() + val df = df0.withColumn("i", toInt(df0("value"))).select("i").toDF().cache() + .filter("i % 2 == 0") + assert(df.collect() === Array(Row(0), Row(4))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + } + + test("Aggregate should be included in WholeStageCodegen with column codegen") { + val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + .groupBy().agg(max(col("value")), avg(col("value"))) + assert(df.collect() === Array(Row(9, 4.5))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + } + + test("Aggregate with grouping keys should be included in WholeStageCodegen with column codegen") { + val df = sparkContext.parallelize(0 to 2, 1).toDF().cache() + .groupBy("value").count().orderBy("value") + assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + } + + test("Aggregate with columns should be included in WholeStageCodegen with column codegen") { + val df = sparkContext.parallelize(0 to 10, 1).map(i => (i, (i * 2).toDouble)).toDF("i", "d") + .cache().agg(sum("d")) + assert(df.collect() === Array(Row(110.0))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + } + + test("Sort should be included in WholeStageCodegen without column codegen") { + val df = sqlContext.range(3, 0, -1).toDF().sort(col("id")) + val plan = df.queryExecution.executedPlan + assert(df.collect() === Array(Row(1), Row(2), Row(3))) + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegen] && + !p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[Sort]).isDefined) + } +} From afdcf042cf112ce3326f6efd3d49759447e490fe Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 21 Apr 2016 07:01:46 -0400 Subject: [PATCH 37/50] remove non-primitive field from CachedBatch to avoid performance overhead in SizeTrackingVector.add at MemoryStore.putIteratorAsValues() --- .../spark/sql/execution/WholeStageCodegenExec.scala | 10 +++++++--- .../execution/columnar/GenerateColumnAccessor.scala | 3 +++ 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index f01d666ee131..eece2e36d533 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -267,12 +267,15 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp ctx.addMutableState(s"$columnarBatchClz", batch, s"$batch = null;", s"$batch = null;") ctx.addMutableState("scala.collection.Iterator", itr, s"$itr = null;", s"$itr = null;") ctx.addMutableState("int", idx, s"$idx = 0;") + val colItr = ctx.freshName("columnarItr") val rowidx = ctx.freshName("rowIdx") val numrows = ctx.freshName("numRows") val colVars = output.indices.map(i => ctx.freshName("col" + i)) val columnAssigns = colVars.zipWithIndex.map { case (name, i) => ctx.addMutableState(columnVectorClz, name, s"$name = null;", s"$name = null;") - s"$name = $batch.column((($columnarItrClz)$input).getColumnIndexes()[$i]);" } + s""" + $name = $batch.column($colItr.getColumnIndexes()[$i], $colItr.getColumnTypes()[$i]); + """ } val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } @@ -280,8 +283,8 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean val incrementReadPartitionAccumulator = if (enableAccumulators) { s""" - (($columnarItrClz)$input).incrementReadPartitionAccumulator(); - """.trim + (($columnarItrClz)$input).incrementReadPartitionAccumulator(); + """.trim } else "" s""" | while (true) { @@ -292,6 +295,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | } | $incrementReadPartitionAccumulator | $batch = ($columnarBatchClz)($itr.next()); + | $columnarItrClz $colItr = ($columnarItrClz)$input; | ${columnAssigns.mkString("", "\n", "")} | } | diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index ae03a7925337..bd60d7bbfe5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -31,6 +31,7 @@ abstract class ColumnarIterator extends Iterator[InternalRow] { columnIndexes: Array[Int], inMemoryColumnarTableScan: InMemoryColumnarTableScan): Unit def getInput: Iterator[CachedBatch] def getColumnIndexes: Array[Int] + def getColumnTypes: Array[DataType] def isSupportColumnarCodeGen: Boolean def incrementReadPartitionAccumulator: Unit } @@ -239,6 +240,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera public int[] getColumnIndexes() { return columnIndexes; } + public DataType[] getColumnTypes() { return columnTypes; } + public boolean isSupportColumnarCodeGen() { return ${_isSupportColumnarCodeGen}; } From e92119964b0daacc93613580151feff6e1f306fb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 22 Apr 2016 07:06:24 -0400 Subject: [PATCH 38/50] simplify Whole stage codegen by moving code for CachedBatch to other files --- .../sql/execution/WholeStageCodegenExec.scala | 70 +++++++------------ .../columnar/GenerateColumnAccessor.scala | 35 +++++++--- .../columnar/InMemoryTableScanExec.scala | 1 + 3 files changed, 50 insertions(+), 56 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index eece2e36d533..842aedc69e8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -258,56 +258,40 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | } """.stripMargin } else { - val columnarBatchClz = "org.apache.spark.sql.execution.columnar.CachedBatch" - val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" - val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" - val batch = WholeStageCodegen.columnarBatchName - val itr = WholeStageCodegen.columnarItrName val idx = WholeStageCodegen.columnarBatchIdxName - ctx.addMutableState(s"$columnarBatchClz", batch, s"$batch = null;", s"$batch = null;") - ctx.addMutableState("scala.collection.Iterator", itr, s"$itr = null;", s"$itr = null;") + val numRows = "columnar_numRows" ctx.addMutableState("int", idx, s"$idx = 0;") - val colItr = ctx.freshName("columnarItr") + ctx.addMutableState("int", numRows, s"$numRows = 0;") val rowidx = ctx.freshName("rowIdx") - val numrows = ctx.freshName("numRows") + val colVars = output.indices.map(i => ctx.freshName("col" + i)) val columnAssigns = colVars.zipWithIndex.map { case (name, i) => - ctx.addMutableState(columnVectorClz, name, s"$name = null;", s"$name = null;") - s""" - $name = $batch.column($colItr.getColumnIndexes()[$i], $colItr.getColumnTypes()[$i]); - """ } - + ctx.addMutableState("org.apache.spark.sql.execution.vectorized.ColumnVector", + name, s"$name = null;", s"$name = null;") + s"$name = ${WholeStageCodegen.columnarItrName}.getColumn($i);" + } val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } - val enableAccumulators: Boolean = - sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean - val incrementReadPartitionAccumulator = if (enableAccumulators) { - s""" - (($columnarItrClz)$input).incrementReadPartitionAccumulator(); - """.trim - } else "" - s""" + + s""" | while (true) { | if ($idx == 0) { - | if (!$itr.hasNext()) { + | $numRows = ${WholeStageCodegen.columnarItrName}.initForColumnar(); + | if ($numRows < 0) { | cleanup(); | break; | } - | $incrementReadPartitionAccumulator - | $batch = ($columnarBatchClz)($itr.next()); - | $columnarItrClz $colItr = ($columnarItrClz)$input; | ${columnAssigns.mkString("", "\n", "")} | } | - | int $numrows = $batch.numRows(); - | while ($idx < $numrows) { + | while ($idx < $numRows) { | int $rowidx = $idx++; | ${consume(ctx, columns, null).trim} | if (shouldStop()) return; | } | $idx = 0; | } - """.stripMargin + """.stripMargin } } @@ -323,8 +307,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp object WholeStageCodegenExec { val PIPELINE_DURATION_METRIC = "duration" - val columnarItrName = "columnar_iterator" - val columnarBatchName = "columnar_batch" + val columnarItrName = "columnar_itr" val columnarBatchIdxName = "columnar_batchIdx" } @@ -378,12 +361,11 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co ctx.isRow = true val codeRow = child.asInstanceOf[CodegenSupport].produce(ctx, this) - val useUnsafeRow = child.find(c => c.isInstanceOf[CodegenSupport] && - c.asInstanceOf[CodegenSupport].useUnsafeRow - ).isDefined - val useInMemoryColumnar = child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]).isDefined - enableColumnCodeGen = ctx.enableColumnCodeGen && !useUnsafeRow && useInMemoryColumnar && - sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean + enableColumnCodeGen = ctx.enableColumnCodeGen && + sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean && + child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]).isDefined && + child.find(c => c.isInstanceOf[CodegenSupport] && + c.asInstanceOf[CodegenSupport].useUnsafeRow).isEmpty val codeProcessNext = if (!enableColumnCodeGen) { s""" @@ -394,9 +376,11 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } else { ctx.isRow = false val codeCol = child.asInstanceOf[CodegenSupport].produce(ctx, this) - val columnItr = ctx.freshName("columnItr") val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" + val colItr = WholeStageCodegen.columnarItrName + ctx.addMutableState(s"$columnarItrClz", colItr, s"$colItr = null;", s"$colItr = null;") + s""" private void processBatch() throws java.io.IOException { ${codeCol.trim} @@ -411,13 +395,9 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } protected void processNext() throws java.io.IOException { - $columnarItrClz $columnItr = null; - if (${WholeStageCodegen.columnarItrName} != null) { - processBatch(); - } else if (${ctx.iteratorInput} instanceof $columnarItrClz && - (($columnItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { - ${WholeStageCodegen.columnarBatchIdxName} = 0; - ${WholeStageCodegen.columnarItrName} = $columnItr.getInput(); + if ((${WholeStageCodegen.columnarBatchIdxName} != 0) || + (${ctx.iteratorInput} instanceof $columnarItrClz && + ($colItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { processBatch(); } else { processRow(); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index bd60d7bbfe5d..ace8d88abeb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.execution.columnar +import scala.collection.Iterator + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, UnsafeRowWriter} +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ /** @@ -29,11 +32,11 @@ import org.apache.spark.sql.types._ abstract class ColumnarIterator extends Iterator[InternalRow] { def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], columnIndexes: Array[Int], inMemoryColumnarTableScan: InMemoryColumnarTableScan): Unit - def getInput: Iterator[CachedBatch] - def getColumnIndexes: Array[Int] - def getColumnTypes: Array[DataType] + def getColumnIndexes(index: Int) : Int + def getColumnTypes(index: Int): DataType def isSupportColumnarCodeGen: Boolean - def incrementReadPartitionAccumulator: Unit + def initForColumnar: Int + def getColumn(index: Int): ColumnVector } /** @@ -166,7 +169,9 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera 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.InMemoryColumnarTableScan; + import org.apache.spark.sql.execution.columnar.CachedBatch; import org.apache.spark.sql.execution.columnar.MutableUnsafeRow; + import org.apache.spark.sql.execution.vectorized.ColumnVector; public SpecificColumnarIterator generate(Object[] references) { return new SpecificColumnarIterator(); @@ -182,6 +187,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera private InMemoryColumnarTableScan inMemoryColumnarTableScan = null; private MutableUnsafeRow mutableRow = null; private boolean readPartitionIncremented = false; + private CachedBatch cachedBatch = null; private int currentRow = 0; private int numRowsInBatch = 0; @@ -216,7 +222,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return false; } - ${classOf[CachedBatch].getName} batch = (${classOf[CachedBatch].getName}) input.next(); + CachedBatch batch = (CachedBatch) input.next(); currentRow = 0; numRowsInBatch = batch.numRows(); for (int i = 0; i < columnIndexes.length; i ++) { @@ -236,21 +242,28 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return unsafeRow; } - public scala.collection.Iterator getInput() { return input; } - - public int[] getColumnIndexes() { return columnIndexes; } + public int getColumnIndexes(int index) { return columnIndexes[index]; } - public DataType[] getColumnTypes() { return columnTypes; } + public DataType getColumnTypes(int index) { return columnTypes[index]; } public boolean isSupportColumnarCodeGen() { return ${_isSupportColumnarCodeGen}; } - public void incrementReadPartitionAccumulator() { - if (!readPartitionIncremented) { + public int initForColumnar() { + if (!input.hasNext()) { + return -1; + } + if ((inMemoryColumnarTableScan != null) && !readPartitionIncremented) { inMemoryColumnarTableScan.incrementReadPartitionAccumulator(); readPartitionIncremented = true; } + cachedBatch = (CachedBatch) input.next(); + return cachedBatch.numRows(); + } + + public ColumnVector getColumn(int i) { + return cachedBatch.column(this, i); } }""" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index d225d324df0a..92025f8ae782 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ From 03a0e9de42a31575385e20826e184effae189d31 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 23 Apr 2016 09:52:44 -0400 Subject: [PATCH 39/50] resolved conflicts --- .../sql/execution/WholeStageCodegenExec.scala | 14 ++++---- .../columnar/GenerateColumnAccessor.scala | 17 ++++----- .../sql/execution/DataFrameCacheSuite.scala | 36 +++++++++---------- 3 files changed, 34 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 842aedc69e8f..b7eefd7be10f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.aggregate.TungstenAggregate -import org.apache.spark.sql.execution.columnar.InMemoryColumnarTableScan +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf @@ -258,7 +258,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | } """.stripMargin } else { - val idx = WholeStageCodegen.columnarBatchIdxName + val idx = WholeStageCodegenExec.columnarBatchIdxName val numRows = "columnar_numRows" ctx.addMutableState("int", idx, s"$idx = 0;") ctx.addMutableState("int", numRows, s"$numRows = 0;") @@ -268,7 +268,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp val columnAssigns = colVars.zipWithIndex.map { case (name, i) => ctx.addMutableState("org.apache.spark.sql.execution.vectorized.ColumnVector", name, s"$name = null;", s"$name = null;") - s"$name = ${WholeStageCodegen.columnarItrName}.getColumn($i);" + s"$name = ${WholeStageCodegenExec.columnarItrName}.getColumn($i);" } val columns = (output zip colVars).map { case (attr, colVar) => new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } @@ -276,7 +276,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp s""" | while (true) { | if ($idx == 0) { - | $numRows = ${WholeStageCodegen.columnarItrName}.initForColumnar(); + | $numRows = ${WholeStageCodegenExec.columnarItrName}.initForColumnar(); | if ($numRows < 0) { | cleanup(); | break; @@ -363,7 +363,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co enableColumnCodeGen = ctx.enableColumnCodeGen && sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean && - child.find(c => c.isInstanceOf[InMemoryColumnarTableScan]).isDefined && + child.find(c => c.isInstanceOf[InMemoryTableScanExec]).isDefined && child.find(c => c.isInstanceOf[CodegenSupport] && c.asInstanceOf[CodegenSupport].useUnsafeRow).isEmpty @@ -378,7 +378,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co val codeCol = child.asInstanceOf[CodegenSupport].produce(ctx, this) val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" - val colItr = WholeStageCodegen.columnarItrName + val colItr = WholeStageCodegenExec.columnarItrName ctx.addMutableState(s"$columnarItrClz", colItr, s"$colItr = null;", s"$colItr = null;") s""" @@ -395,7 +395,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } protected void processNext() throws java.io.IOException { - if ((${WholeStageCodegen.columnarBatchIdxName} != 0) || + if ((${WholeStageCodegenExec.columnarBatchIdxName} != 0) || (${ctx.iteratorInput} instanceof $columnarItrClz && ($colItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { processBatch(); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index ace8d88abeb0..bee1592ccd50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -22,7 +22,8 @@ import scala.collection.Iterator import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, UnsafeRowWriter} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator, UnsafeRowWriter} +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ @@ -31,7 +32,7 @@ import org.apache.spark.sql.types._ */ abstract class ColumnarIterator extends Iterator[InternalRow] { def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], - columnIndexes: Array[Int], inMemoryColumnarTableScan: InMemoryColumnarTableScan): Unit + columnIndexes: Array[Int], inMemoryTableScanExec: InMemoryTableScanExec): Unit def getColumnIndexes(index: Int) : Int def getColumnTypes(index: Int): DataType def isSupportColumnarCodeGen: Boolean @@ -168,7 +169,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera import org.apache.spark.sql.types.DataType; 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.InMemoryColumnarTableScan; + import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec; import org.apache.spark.sql.execution.columnar.CachedBatch; import org.apache.spark.sql.execution.columnar.MutableUnsafeRow; import org.apache.spark.sql.execution.vectorized.ColumnVector; @@ -184,7 +185,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera private UnsafeRow unsafeRow = new UnsafeRow($numFields); private BufferHolder bufferHolder = new BufferHolder(unsafeRow); private UnsafeRowWriter rowWriter = new UnsafeRowWriter(bufferHolder, $numFields); - private InMemoryColumnarTableScan inMemoryColumnarTableScan = null; + private InMemoryTableScanExec inMemoryTableScanExec = null; private MutableUnsafeRow mutableRow = null; private boolean readPartitionIncremented = false; private CachedBatch cachedBatch = null; @@ -205,11 +206,11 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } public void initialize(Iterator input, DataType[] columnTypes, - int[] columnIndexes, InMemoryColumnarTableScan inMemoryColumnarTableScan) { + int[] columnIndexes, InMemoryTableScanExec inMemoryTableScanExec) { this.input = input; this.columnTypes = columnTypes; this.columnIndexes = columnIndexes; - this.inMemoryColumnarTableScan = inMemoryColumnarTableScan; + this.inMemoryTableScanExec = inMemoryTableScanExec; } ${ctx.declareAddedFunctions()} @@ -254,8 +255,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera if (!input.hasNext()) { return -1; } - if ((inMemoryColumnarTableScan != null) && !readPartitionIncremented) { - inMemoryColumnarTableScan.incrementReadPartitionAccumulator(); + if ((inMemoryTableScanExec != null) && !readPartitionIncremented) { + inMemoryTableScanExec.incrementReadPartitionAccumulator(); readPartitionIncremented = true; } cachedBatch = (CachedBatch) input.next(); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala index 3fb7d5f4d195..a3576cc6191a 100755 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala @@ -32,8 +32,8 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(df.collect() === Array(Row(2))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen).isDefined) } test("filters should be combined with column codegen") { @@ -42,8 +42,8 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(df.collect() === Array(Row(0), Row(6))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen).isDefined) } test("filter with null should be included in WholeStageCodegen with column codegen") { @@ -54,8 +54,8 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(df.collect() === Array(Row(0), Row(4))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen).isDefined) } test("Aggregate should be included in WholeStageCodegen with column codegen") { @@ -64,9 +64,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(df.collect() === Array(Row(9, 4.5))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) } test("Aggregate with grouping keys should be included in WholeStageCodegen with column codegen") { @@ -75,9 +75,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) } test("Aggregate with columns should be included in WholeStageCodegen with column codegen") { @@ -86,9 +86,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(df.collect() === Array(Row(110.0))) val plan = df.queryExecution.executedPlan assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[TungstenAggregate]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) } test("Sort should be included in WholeStageCodegen without column codegen") { @@ -96,8 +96,8 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { val plan = df.queryExecution.executedPlan assert(df.collect() === Array(Row(1), Row(2), Row(3))) assert(plan.find(p => - p.isInstanceOf[WholeStageCodegen] && - !p.asInstanceOf[WholeStageCodegen].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[Sort]).isDefined) + p.isInstanceOf[WholeStageCodegenExec] && + !p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec]).isDefined) } } From 06a665fd0aa397cb35901832cacea6f24be574d5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 23 Apr 2016 18:29:59 -0400 Subject: [PATCH 40/50] fix build error --- .../spark/sql/execution/columnar/GenerateColumnAccessor.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index bee1592ccd50..52edf5f4466a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -23,7 +23,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator, UnsafeRowWriter} -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ From 116e78aa7ff1d9f1b089c1be8d8f54e434d32800 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 4 May 2016 04:27:58 -0400 Subject: [PATCH 41/50] Supprt only float/double Move to support for boolean/byte/short/int/long into another PR to reduce size of this PR for ease of review --- .../vectorized/ByteBufferColumnVector.java | 67 ++- .../sql/execution/columnar/ColumnType.scala | 18 - .../columnar/GenerateColumnAccessor.scala | 5 + .../compression/compressionSchemes.scala | 279 +------------ .../spark/sql/DataFrameCacheBenchmark.scala | 384 ++++++----------- .../sql/execution/DataFrameCacheSuite.scala | 28 +- .../compression/BooleanBitSetSuite.scala | 51 --- .../compression/DictionaryEncodingSuite.scala | 74 ---- .../compression/IntegralDeltaSuite.scala | 77 ---- .../compression/PassThroughSuite.scala | 392 +++++++++--------- .../compression/RunLengthEncodingSuite.scala | 90 ---- 11 files changed, 366 insertions(+), 1099 deletions(-) mode change 100755 => 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java index e385f6e4a281..e8cca482ddc4 100755 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java @@ -49,12 +49,7 @@ protected ByteBufferColumnVector(int capacity, DataType type, super(capacity, type, MemoryMode.ON_HEAP); if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { throw new NotImplementedException(); - } else if (type instanceof BooleanType || type instanceof ByteType || - type instanceof ShortType || - type instanceof IntegerType || type instanceof DateType || - DecimalType.is32BitDecimalType(type) || - type instanceof LongType || DecimalType.is64BitDecimalType(type) || - (type instanceof FloatType) || (type instanceof DoubleType)) { + } else if ((type instanceof FloatType) || (type instanceof DoubleType)) { data = buffer.array(); offset = Platform.BYTE_ARRAY_OFFSET + buffer.position(); } else if (resultStruct != null) { @@ -132,49 +127,41 @@ public final boolean isNullAt(int rowId) { @Override public final void putBoolean(int rowId, boolean value) { - Platform.putByte(data, offset + rowId, (byte)((value) ? 1 : 0)); + throw new NotImplementedException(); } @Override public final void putBooleans(int rowId, int count, boolean value) { - byte v = (byte)((value) ? 1 : 0); - for (int i = 0; i < count; ++i) { - Platform.putByte(data, offset + i + rowId, v); - } + throw new NotImplementedException(); } @Override public final boolean getBoolean(int rowId) { - return Platform.getByte(data, offset + rowId) == 1; + throw new NotImplementedException(); } - // - // // APIs dealing with Bytes // @Override public final void putByte(int rowId, byte value) { - Platform.putByte(data, offset + rowId, value); + throw new NotImplementedException(); } @Override public final void putBytes(int rowId, int count, byte value) { - for (int i = 0; i < count; ++i) { - Platform.putByte(data, offset + i + rowId, value); - } + throw new NotImplementedException(); } @Override public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { - System.arraycopy(src, srcIndex, data, rowId, count); + throw new NotImplementedException(); } @Override public final byte getByte(int rowId) { - assert(dictionary == null); - return Platform.getByte(data, offset + rowId); + throw new NotImplementedException(); } // @@ -183,14 +170,12 @@ public final byte getByte(int rowId) { @Override public final void putShort(int rowId, short value) { - Platform.putShort(data, offset + rowId * 2, value); + throw new NotImplementedException(); } @Override public final void putShorts(int rowId, int count, short value) { - for (int i = 0; i < count; ++i) { - Platform.putShort(data, offset + (i + rowId) * 2, value); - } + throw new NotImplementedException(); } @Override @@ -200,8 +185,7 @@ public final void putShorts(int rowId, int count, short[] src, int srcIndex) { @Override public final short getShort(int rowId) { - assert(dictionary == null); - return Platform.getShort(data, offset + rowId * 2); + throw new NotImplementedException(); } @@ -211,14 +195,12 @@ public final short getShort(int rowId) { @Override public final void putInt(int rowId, int value) { - Platform.putInt(data, offset + rowId * 4, value); + throw new NotImplementedException(); } @Override public final void putInts(int rowId, int count, int value) { - for (int i = 0; i < count; ++i) { - Platform.putInt(data, offset + (i + rowId) * 4, value); - } + throw new NotImplementedException(); } @Override @@ -233,8 +215,7 @@ public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcI @Override public final int getInt(int rowId) { - assert(dictionary == null); - return Platform.getInt(data, offset + rowId * 4); + throw new NotImplementedException(); } // @@ -243,14 +224,12 @@ public final int getInt(int rowId) { @Override public final void putLong(int rowId, long value) { - Platform.putLong(data, offset + rowId * 8, value); + throw new NotImplementedException(); } @Override public final void putLongs(int rowId, int count, long value) { - for (int i = 0; i < count; ++i) { - Platform.putLong(data, offset + (i + rowId) * 8, value); - } + throw new NotImplementedException(); } @Override @@ -265,8 +244,7 @@ public final void putLongsLittleEndian(int rowId, int count, byte[] src, int src @Override public final long getLong(int rowId) { - assert(dictionary == null); - return Platform.getLong(data, offset + rowId * 8); + throw new NotImplementedException(); } // @@ -274,7 +252,9 @@ public final long getLong(int rowId) { // @Override - public final void putFloat(int rowId, float value) { Platform.putFloat(data, offset + rowId * 4, value); } + public final void putFloat(int rowId, float value) { + throw new NotImplementedException(); + } @Override public final void putFloats(int rowId, int count, float value) { @@ -288,8 +268,7 @@ public final void putFloats(int rowId, int count, float[] src, int srcIndex) { @Override public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - data, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + throw new NotImplementedException(); } @Override @@ -303,7 +282,9 @@ public final float getFloat(int rowId) { // @Override - public final void putDouble(int rowId, double value) { Platform.putDouble(data, offset + rowId * 8, value); } + public final void putDouble(int rowId, double value) { + throw new NotImplementedException(); + } @Override public final void putDoubles(int rowId, int count, double value) { 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 23ec157ac531..2d47e84f9cf3 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 @@ -73,24 +73,6 @@ private[columnar] object ByteBufferHelper { 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() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 52edf5f4466a..fc05a555876b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -101,6 +101,11 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera val createCode = dt match { case t if ctx.isPrimitiveType(dt) => + dt match { + case FloatType => + case DoubleType => + case _ => _isSupportColumnarCodeGen = false + } s"$accessorName = new $accessorCls(ByteBuffer.wrap(buffers[$index]).order(nativeOrder));" case NullType | StringType | BinaryType => _isSupportColumnarCodeGen = false 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 556a36512f50..4a253dea6447 100755 --- 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 @@ -72,11 +72,6 @@ private[columnar] case object PassThrough extends CompressionScheme { (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.") @@ -217,142 +212,7 @@ 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.") - } + throw new IllegalStateException("Not support in RunLengthEncoding.") } } } @@ -465,51 +325,7 @@ 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.") - } + throw new IllegalStateException("Not support in DictionaryEncoding.") } } } @@ -603,39 +419,7 @@ 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) + throw new IllegalStateException("Not support in BooleanBitSet") } } } @@ -719,34 +503,7 @@ private[columnar] case object IntDelta extends CompressionScheme { } 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) + throw new IllegalStateException("Not support in IntDelta") } } } @@ -830,33 +587,7 @@ private[columnar] case object LongDelta extends CompressionScheme { } 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) + throw new IllegalStateException("Not support in LongDelta") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala index fd1ee8618e8d..d09abd4781db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala @@ -1,262 +1,122 @@ -/* - * 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 - -import java.util.Random - -import scala.util.Try - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql._ -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.util.Benchmark - -/** - * Benchmark to measure performance of columnar storage for dataframe cache. - * To run this: - * spark-submit --class - */ -object DataFrameCacheBenchmark { - val conf = new SparkConf() - val sc = new SparkContext("local[1]", "test-sql-context", conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // Set default configs. Individual cases will change them if necessary. - sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - - def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { - val (keys, values) = pairs.unzip - val currentValues = keys.map(key => Try(sqlContext.conf.getConfString(key)).toOption) - (keys, values).zipped.foreach(sqlContext.conf.setConfString) - try f finally { - keys.zip(currentValues).foreach { - case (key, Some(value)) => sqlContext.conf.setConfString(key, value) - case (key, None) => sqlContext.conf.unsetConf(key) - } - } - } - - def intSumBenchmark(values: Int, iters: Int = 5): Unit = { - val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) - - val benchmarkPT = new Benchmark("Int Sum with PassThrough cache", values, iters) - val rand1 = new Random(511) - val dfPassThrough = sc.parallelize(0 to values - 1, 1).map(i => rand1.nextInt()).toDF.cache() - dfPassThrough.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkPT.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfPassThrough.agg(sum("value")).collect - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Int Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 462 / 466 68.1 14.7 1.0X - ColumnVector codegen 94 / 100 336.3 3.0 4.9X - */ - - benchmarkPT.run() - dfPassThrough.unpersist(true) - System.gc() - - val benchmarkRL = new Benchmark("Int Sum with RunLength cache", values, iters) - val dfRunLength = sc.parallelize(0 to values - 1, 1) - .map(i => (i, (i / 1024).toInt)).toDF("k", "v").cache() - dfRunLength.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkRL.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfRunLength.agg(sum("v")).collect() - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Int Sum with RunLength cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 492 / 553 63.9 15.7 1.0X - ColumnVector codegen 175 / 180 179.5 5.6 2.8X - */ - - benchmarkRL.run() - dfRunLength.unpersist(true) - System.gc() - - val rand2 = new Random(42) - val benchmarkDIC = new Benchmark("Int Sum with Dictionary cache", values, iters) - val dfDictionary = sc.parallelize(0 to values - 1, 1) - .map(i => (i, (rand2.nextInt() % 1023))).toDF("k", "v").cache() - dfDictionary.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkDIC.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfDictionary.agg(sum("v")).collect() - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Int Sum with Dictionary cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 599 / 610 52.5 19.0 1.0X - ColumnVector codegen 320 / 327 98.3 10.2 1.9X - */ - - benchmarkDIC.run() - dfDictionary.unpersist(true) - System.gc() - - val benchmarkIDelta = new Benchmark("Int Sum with Delta cache", values, iters) - val dfIntDelta = sc.parallelize(0 to values - 1, 1) - .map(i => (i, i)).toDF("k", "v").cache() - dfIntDelta.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkIDelta.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfIntDelta.agg(sum("v")).collect() - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Int Sum with Delta cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 467 / 512 67.4 14.8 1.0X - ColumnVector codegen 247 / 250 127.4 7.8 1.9X - */ - - benchmarkIDelta.run() - dfIntDelta.unpersist(true) - System.gc() - } - - def longSumBenchmark(values: Int, iters: Int = 5): Unit = { - val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) - - val benchmarkPT = new Benchmark("Long Sum with PassThrough cache", values, iters) - val rand1 = new Random(511) - val dfPassThrough = sc.parallelize(0 to values - 1, 1).map(i => rand1.nextLong()).toDF().cache() - dfPassThrough.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkPT.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfPassThrough.agg(sum("value")).collect - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Long Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 382 / 420 41.2 24.3 1.0X - ColumnVector codegen 89 / 101 176.2 5.7 4.3X - */ - - benchmarkPT.run() - dfPassThrough.unpersist(true) - System.gc() - } - - def floatSumBenchmark(values: Int, iters: Int = 5): Unit = { - val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) - - val benchmarkPT = new Benchmark("Float Sum with PassThrough cache", values, iters) - val rand1 = new Random(511) - val dfPassThrough = sc.parallelize(0 to values - 1, 1) - .map(i => rand1.nextFloat()).toDF().cache() - dfPassThrough.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkPT.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfPassThrough.agg(sum("value")).collect - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Float Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 476 / 483 66.1 15.1 1.0X - ColumnVector codegen 91 / 103 343.8 2.9 5.2X - */ - - benchmarkPT.run() - dfPassThrough.unpersist(true) - System.gc() - } - - def doubleSumBenchmark(values: Int, iters: Int = 5): Unit = { - val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) - - val benchmarkPT = new Benchmark("Double Sum with PassThrough cache", values, iters) - val rand1 = new Random(511) - val dfPassThrough = sc.parallelize(0 to values - 1, 1) - .map(i => rand1.nextDouble()).toDF().cache() - dfPassThrough.count() // force to create df.cache() - suites.foreach { - case (str, value) => - benchmarkPT.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { - dfPassThrough.agg(sum("value")).collect - } - } - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 - Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz - Double Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - InternalRow codegen 290 / 306 54.3 18.4 1.0X - ColumnVector codegen 95 / 101 165.7 6.0 3.1X - */ - - benchmarkPT.run() - dfPassThrough.unpersist(true) - System.gc() - } - - def main(args: Array[String]): Unit = { - longSumBenchmark(1024 * 1024 * 15) - doubleSumBenchmark(1024 * 1024 * 15) - floatSumBenchmark(1024 * 1024 * 30) - intSumBenchmark(1024 * 1024 * 30) - } -} +/* + * 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 + +import java.util.Random + +import scala.util.Try + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure performance of columnar storage for dataframe cache. + * To run this: + * spark-submit --class + */ +object DataFrameCacheBenchmark { + val conf = new SparkConf() + val sc = new SparkContext("local[1]", "test-sql-context", conf) + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Set default configs. Individual cases will change them if necessary. + sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + + def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(sqlContext.conf.getConfString(key)).toOption) + (keys, values).zipped.foreach(sqlContext.conf.setConfString) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => sqlContext.conf.setConfString(key, value) + case (key, None) => sqlContext.conf.unsetConf(key) + } + } + } + + def floatSumBenchmark(values: Int, iters: Int = 5): Unit = { + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) + + val benchmarkPT = new Benchmark("Float Sum with PassThrough cache", values, iters) + val rand1 = new Random(511) + val dfPassThrough = sc.parallelize(0 to values - 1, 1) + .map(i => rand1.nextFloat()).toDF().cache() + dfPassThrough.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkPT.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfPassThrough.agg(sum("value")).collect + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Float Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 476 / 483 66.1 15.1 1.0X + ColumnVector codegen 91 / 103 343.8 2.9 5.2X + */ + + benchmarkPT.run() + dfPassThrough.unpersist(true) + System.gc() + } + + def doubleSumBenchmark(values: Int, iters: Int = 5): Unit = { + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) + + val benchmarkPT = new Benchmark("Double Sum with PassThrough cache", values, iters) + val rand1 = new Random(511) + val dfPassThrough = sc.parallelize(0 to values - 1, 1) + .map(i => rand1.nextDouble()).toDF().cache() + dfPassThrough.count() // force to create df.cache() + suites.foreach { + case (str, value) => + benchmarkPT.addCase(s"$str codegen") { iter => + withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + dfPassThrough.agg(sum("value")).collect + } + } + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 + Intel(R) Xeon(R) CPU E5-2667 v2 @ 3.30GHz + Double Sum with PassThrough cache: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + InternalRow codegen 290 / 306 54.3 18.4 1.0X + ColumnVector codegen 95 / 101 165.7 6.0 3.1X + */ + + benchmarkPT.run() + dfPassThrough.unpersist(true) + System.gc() + } + + def main(args: Array[String]): Unit = { + doubleSumBenchmark(1024 * 1024 * 15) + floatSumBenchmark(1024 * 1024 * 30) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala old mode 100755 new mode 100644 index a3576cc6191a..133cc392fbc2 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala @@ -27,9 +27,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("range/filter should be combined with column codegen") { - val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + val df = sparkContext.parallelize(0 to 9, 1).map(i => i.toFloat).toDF().cache() .filter("value = 1").selectExpr("value + 1") - assert(df.collect() === Array(Row(2))) + assert(df.collect() === Array(Row(2.0))) val plan = df.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && @@ -37,9 +37,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { } test("filters should be combined with column codegen") { - val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() - .filter("value % 2 == 0").filter("value % 3 == 0") - assert(df.collect() === Array(Row(0), Row(6))) + val df = sparkContext.parallelize(0 to 9, 1).map(i => i.toFloat).toDF().cache() + .filter("value % 2.0 == 0").filter("value % 3.0 == 0") + assert(df.collect() === Array(Row(0), Row(6.0))) val plan = df.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && @@ -47,11 +47,11 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { } test("filter with null should be included in WholeStageCodegen with column codegen") { - val toInt = udf[java.lang.Integer, String] { s => if (s == "2") null else s.toInt } + val toFloat = udf[java.lang.Float, String] { s => if (s == "2") null else s.toFloat } val df0 = sparkContext.parallelize(0 to 4, 1).map(i => i.toString).toDF() - val df = df0.withColumn("i", toInt(df0("value"))).select("i").toDF().cache() - .filter("i % 2 == 0") - assert(df.collect() === Array(Row(0), Row(4))) + val df = df0.withColumn("i", toFloat(df0("value"))).select("i").toDF().cache() + .filter("i % 2.0 == 0") + assert(df.collect() === Array(Row(0), Row(4.0))) val plan = df.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && @@ -59,7 +59,7 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { } test("Aggregate should be included in WholeStageCodegen with column codegen") { - val df = sparkContext.parallelize(0 to 9, 1).toDF().cache() + val df = sparkContext.parallelize(0 to 9, 1).map(i => i.toFloat).toDF().cache() .groupBy().agg(max(col("value")), avg(col("value"))) assert(df.collect() === Array(Row(9, 4.5))) val plan = df.queryExecution.executedPlan @@ -70,9 +70,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { } test("Aggregate with grouping keys should be included in WholeStageCodegen with column codegen") { - val df = sparkContext.parallelize(0 to 2, 1).toDF().cache() + val df = sparkContext.parallelize(0 to 2, 1).map(i => i.toFloat).toDF().cache() .groupBy("value").count().orderBy("value") - assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) + assert(df.collect() === Array(Row(0.0, 1), Row(1.0, 1), Row(2.0, 1))) val plan = df.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && @@ -92,9 +92,9 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { } test("Sort should be included in WholeStageCodegen without column codegen") { - val df = sqlContext.range(3, 0, -1).toDF().sort(col("id")) + val df = sqlContext.range(3, 0, -1).map(i => i.toFloat).toDF().sort(col("value")) val plan = df.queryExecution.executedPlan - assert(df.collect() === Array(Row(1), Row(2), Row(3))) + assert(df.collect() === Array(Row(1.0), Row(2.0), Row(3.0))) assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && !p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && 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 08390d6cdb15..f67e9c7dae27 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,37 +85,6 @@ 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) } @@ -135,24 +104,4 @@ 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 84a138f8586e..b0b611560eac 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 @@ -115,66 +115,6 @@ 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 GenericMutableRow(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) } @@ -186,19 +126,5 @@ 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 405f2e0463da..6d73c79b92e6 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 @@ -110,61 +110,6 @@ class IntegralDeltaSuite extends SparkFunSuite { assert(!decoder.hasNext) } - def skeletonForDecompress(input: Seq[I#InternalType]) { - val builder = TestCompressibleColumnBuilder(columnStats, columnType, scheme) - val row = new GenericMutableRow(1) - val nullRow = new GenericMutableRow(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) } @@ -183,27 +128,5 @@ 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/PassThroughSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala index c9a9da9b132b..c160cfdc9f4e 100755 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala @@ -1,196 +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.GenericMutableRow -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 GenericMutableRow(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 GenericMutableRow(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 GenericMutableRow(1) - val nullRow = new GenericMutableRow(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])) - } - } -} +/* + * 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.GenericMutableRow +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 GenericMutableRow(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 GenericMutableRow(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 GenericMutableRow(1) + val nullRow = new GenericMutableRow(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])) + } + } +} 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 86ef34f3cd83..76156e508053 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 @@ -97,80 +97,6 @@ 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 GenericMutableRow(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) } @@ -186,21 +112,5 @@ 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)) - } } } From a12f3ca09a4830341621bfe93f96757b44d128ac Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 4 May 2016 06:04:04 -0400 Subject: [PATCH 42/50] fix test failures --- .../execution/columnar/compression/PassThroughSuite.scala | 8 -------- 1 file changed, 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala index c160cfdc9f4e..70d0590eabcd 100755 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala @@ -25,10 +25,6 @@ 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) @@ -182,10 +178,6 @@ class PassThroughSuite extends SparkFunSuite { 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) } From ff3e77087ed6f1aa0c6db29a8f2a74f994a6e6ae Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 8 May 2016 15:37:03 -0400 Subject: [PATCH 43/50] move most of implementations into InMemoryTableScanExec --- .../sql/execution/WholeStageCodegenExec.scala | 89 +++--------------- .../columnar/InMemoryTableScanExec.scala | 91 ++++++++++++++++++- 2 files changed, 102 insertions(+), 78 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index b7eefd7be10f..f2f6386dc1bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -55,7 +55,11 @@ trait CodegenSupport extends SparkPlan { * @return name of the variable representing the metric */ def metricTerm(ctx: CodegenContext, name: String): String = { - ctx.addReferenceObj(name, longMetric(name)) + val metric = ctx.addReferenceObj(name, longMetric(name)) + val value = ctx.freshName("metricValue") + val cls = classOf[LongSQLMetricValue].getName + ctx.addMutableState(cls, value, s"$value = ($cls) $metric.localValue();") + value } /** @@ -113,7 +117,7 @@ trait CodegenSupport extends SparkPlan { protected def doProduce(ctx: CodegenContext): String /** - * Consume the generated columns or row from current SparkPlan, call its parent's `doConsume()`. + * Consume the generated columns or row from current SparkPlan, call it's parent's doConsume(). */ final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { val inputVars = @@ -220,8 +224,8 @@ trait CodegenSupport extends SparkPlan { /** * InputAdapter is used to hide a SparkPlan from a subtree that support codegen. * - * This is the leaf node of a tree with WholeStageCodegen that is used to generate code - * that consumes an RDD iterator of InternalRow. + * This is the leaf node of a tree with WholeStageCodegen, is used to generate code that consumes + * an RDD iterator of InternalRow. */ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { @@ -258,40 +262,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp | } """.stripMargin } else { - val idx = WholeStageCodegenExec.columnarBatchIdxName - val numRows = "columnar_numRows" - ctx.addMutableState("int", idx, s"$idx = 0;") - ctx.addMutableState("int", numRows, s"$numRows = 0;") - val rowidx = ctx.freshName("rowIdx") - - val colVars = output.indices.map(i => ctx.freshName("col" + i)) - val columnAssigns = colVars.zipWithIndex.map { case (name, i) => - ctx.addMutableState("org.apache.spark.sql.execution.vectorized.ColumnVector", - name, s"$name = null;", s"$name = null;") - s"$name = ${WholeStageCodegenExec.columnarItrName}.getColumn($i);" - } - val columns = (output zip colVars).map { case (attr, colVar) => - new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } - - s""" - | while (true) { - | if ($idx == 0) { - | $numRows = ${WholeStageCodegenExec.columnarItrName}.initForColumnar(); - | if ($numRows < 0) { - | cleanup(); - | break; - | } - | ${columnAssigns.mkString("", "\n", "")} - | } - | - | while ($idx < $numRows) { - | int $rowidx = $idx++; - | ${consume(ctx, columns, null).trim} - | if (shouldStop()) return; - | } - | $idx = 0; - | } - """.stripMargin + InMemoryTableScanExec.produceColumnLoop(ctx, this, output) } } @@ -307,8 +278,6 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp object WholeStageCodegenExec { val PIPELINE_DURATION_METRIC = "duration" - val columnarItrName = "columnar_itr" - val columnarBatchIdxName = "columnar_batchIdx" } /** @@ -361,12 +330,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co ctx.isRow = true val codeRow = child.asInstanceOf[CodegenSupport].produce(ctx, this) - enableColumnCodeGen = ctx.enableColumnCodeGen && - sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean && - child.find(c => c.isInstanceOf[InMemoryTableScanExec]).isDefined && - child.find(c => c.isInstanceOf[CodegenSupport] && - c.asInstanceOf[CodegenSupport].useUnsafeRow).isEmpty - + enableColumnCodeGen = InMemoryTableScanExec.enableColumnCodeGen(sqlContext, ctx, child) val codeProcessNext = if (!enableColumnCodeGen) { s""" protected void processNext() throws java.io.IOException { @@ -374,36 +338,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } """ } else { - ctx.isRow = false - val codeCol = child.asInstanceOf[CodegenSupport].produce(ctx, this) - - val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" - val colItr = WholeStageCodegenExec.columnarItrName - ctx.addMutableState(s"$columnarItrClz", colItr, s"$colItr = null;", s"$colItr = null;") - - s""" - private void processBatch() throws java.io.IOException { - ${codeCol.trim} - } - - private void processRow() throws java.io.IOException { - ${codeRow.trim} - } - - private void cleanup() { - ${ctx.cleanupMutableStates()} - } - - protected void processNext() throws java.io.IOException { - if ((${WholeStageCodegenExec.columnarBatchIdxName} != 0) || - (${ctx.iteratorInput} instanceof $columnarItrClz && - ($colItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { - processBatch(); - } else { - processRow(); - } - } - """.trim + InMemoryTableScanExec.produceProcessNext(ctx, this, child, codeRow) } val source = s""" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 92025f8ae782..c5f814d31777 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -18,14 +18,18 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -38,7 +42,7 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren private[sql] override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes @@ -187,3 +191,88 @@ private[sql] case class InMemoryTableScanExec( } } } + +private[sql] object InMemoryTableScanExec { + private val columnarItrName = "columnar_itr" + private val columnarBatchIdxName = "columnar_batchIdx" + + def enableColumnCodeGen( + sqlContext: SQLContext, ctx: CodegenContext, child: SparkPlan): Boolean = { + ctx.enableColumnCodeGen && + sqlContext.getConf(SQLConf.COLUMN_VECTOR_CODEGEN.key).toBoolean && + child.find(c => c.isInstanceOf[InMemoryTableScanExec]).isDefined && + child.find(c => c.isInstanceOf[CodegenSupport] && + c.asInstanceOf[CodegenSupport].useUnsafeRow).isEmpty + } + + def produceColumnLoop( + ctx: CodegenContext, codegen: CodegenSupport, output: Seq[Attribute]): String = { + val idx = columnarBatchIdxName + val numRows = "columnar_numRows" + ctx.addMutableState("int", idx, s"$idx = 0;") + ctx.addMutableState("int", numRows, s"$numRows = 0;") + val rowidx = ctx.freshName("rowIdx") + + val colVars = output.indices.map(i => ctx.freshName("col" + i)) + val columnAssigns = colVars.zipWithIndex.map { case (name, i) => + ctx.addMutableState("org.apache.spark.sql.execution.vectorized.ColumnVector", + name, s"$name = null;", s"$name = null;") + s"$name = ${columnarItrName}.getColumn($i);" + } + val columns = (output zip colVars).map { case (attr, colVar) => + new ColumnVectorReference(colVar, rowidx, attr.dataType, attr.nullable).genCode(ctx) } + + s""" + | while (true) { + | if ($idx == 0) { + | $numRows = ${columnarItrName}.initForColumnar(); + | if ($numRows < 0) { + | cleanup(); + | break; + | } + | ${columnAssigns.mkString("", "\n", "")} + | } + | + | while ($idx < $numRows) { + | int $rowidx = $idx++; + | ${codegen.consume(ctx, columns, null).trim} + | if (shouldStop()) return; + | } + | $idx = 0; + | } + """.stripMargin + } + + def produceProcessNext( + ctx: CodegenContext, codegen: CodegenSupport, child: SparkPlan, codeRow: String): String = { + ctx.isRow = false + val codeCol = child.asInstanceOf[CodegenSupport].produce(ctx, codegen) + val columnarItrClz = "org.apache.spark.sql.execution.columnar.ColumnarIterator" + val colItr = columnarItrName + ctx.addMutableState(s"$columnarItrClz", colItr, s"$colItr = null;", s"$colItr = null;") + + s""" + private void processBatch() throws java.io.IOException { + ${codeCol.trim} + } + + private void processRow() throws java.io.IOException { + ${codeRow.trim} + } + + private void cleanup() { + ${ctx.cleanupMutableStates()} + } + + protected void processNext() throws java.io.IOException { + if ((${columnarBatchIdxName} != 0) || + (${ctx.iteratorInput} instanceof $columnarItrClz && + ($colItr = ($columnarItrClz)${ctx.iteratorInput}).isSupportColumnarCodeGen())) { + processBatch(); + } else { + processRow(); + } + } + """.trim + } +} From 6ec8d6ad297b002533fc7c46e457c9ed8d2deac3 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 8 May 2016 22:52:53 -0400 Subject: [PATCH 44/50] fix build error --- .../sql/execution/WholeStageCodegenExec.scala | 14 +++++--------- .../execution/columnar/InMemoryTableScanExec.scala | 2 +- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index f2f6386dc1bb..810634b1f765 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} -import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -55,11 +55,7 @@ trait CodegenSupport extends SparkPlan { * @return name of the variable representing the metric */ def metricTerm(ctx: CodegenContext, name: String): String = { - val metric = ctx.addReferenceObj(name, longMetric(name)) - val value = ctx.freshName("metricValue") - val cls = classOf[LongSQLMetricValue].getName - ctx.addMutableState(cls, value, s"$value = ($cls) $metric.localValue();") - value + ctx.addReferenceObj(name, longMetric(name)) } /** @@ -117,7 +113,7 @@ trait CodegenSupport extends SparkPlan { protected def doProduce(ctx: CodegenContext): String /** - * Consume the generated columns or row from current SparkPlan, call it's parent's doConsume(). + * Consume the generated columns or row from current SparkPlan, call its parent's `doConsume()`. */ final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { val inputVars = @@ -224,8 +220,8 @@ trait CodegenSupport extends SparkPlan { /** * InputAdapter is used to hide a SparkPlan from a subtree that support codegen. * - * This is the leaf node of a tree with WholeStageCodegen, is used to generate code that consumes - * an RDD iterator of InternalRow. + * This is the leaf node of a tree with WholeStageCodegen that is used to generate code + * that consumes an RDD iterator of InternalRow. */ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index c5f814d31777..ee64a3b188e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -42,7 +42,7 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren private[sql] override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes From 9a02ce54824799adf194a161dca722eacb13f12f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 10 Jun 2016 03:48:57 -0400 Subject: [PATCH 45/50] rebase --- .../sql/execution/vectorized/ByteBufferColumnVector.java | 0 .../spark/sql/execution/WholeStageCodegenExec.scala | 2 -- .../sql/execution/columnar/GenerateColumnAccessor.scala | 2 +- .../sql/execution/columnar/InMemoryTableScanExec.scala | 5 ++--- .../columnar/compression/compressionSchemes.scala | 0 .../apache/spark/sql/execution/DataFrameCacheSuite.scala | 8 ++++---- .../execution/columnar/compression/PassThroughSuite.scala | 0 7 files changed, 7 insertions(+), 10 deletions(-) mode change 100755 => 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java mode change 100755 => 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala mode change 100755 => 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java old mode 100755 new mode 100644 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 810634b1f765..a7dc93f7221a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -24,9 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index fc05a555876b..b62696ee695d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -22,7 +22,7 @@ import scala.collection.Iterator import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator, UnsafeRowWriter} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, UnsafeRowWriter} import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index ee64a3b188e3..dcedc542014a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -22,10 +22,9 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized.ColumnVector @@ -116,7 +115,7 @@ private[sql] case class InMemoryTableScanExec( lazy val readBatches = sparkContext.longAccumulator def incrementReadPartitionAccumulator(): Unit = { - readPartitions += 1 + readPartitions.add(1) } private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning 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 old mode 100755 new mode 100644 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala index 133cc392fbc2..6f36707359ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{QueryTest, Row} -import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -66,7 +66,7 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) } test("Aggregate with grouping keys should be included in WholeStageCodegen with column codegen") { @@ -77,7 +77,7 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) } test("Aggregate with columns should be included in WholeStageCodegen with column codegen") { @@ -88,7 +88,7 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && p.asInstanceOf[WholeStageCodegenExec].enableColumnCodeGen && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) } test("Sort should be included in WholeStageCodegen without column codegen") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughSuite.scala old mode 100755 new mode 100644 From 697890cfc2cc32944422bafe121c928a1ac3051d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 29 Jun 2016 18:37:21 +0900 Subject: [PATCH 46/50] rebase --- .../vectorized/ByteBufferColumnVector.java | 3 ++- .../execution/columnar/InMemoryRelation.scala | 23 +++++++++---------- .../sql/execution/DataFrameCacheSuite.scala | 3 ++- 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java index e8cca482ddc4..5d35179c96d2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java @@ -341,7 +341,8 @@ public final void reserve(int requiredCapacity) { } // Spilt this function out since it is the slow path. - private final void reserveInternal(int newCapacity) { + @Override + protected void reserveInternal(int newCapacity) { throw new NotImplementedException(); } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index a1c64e9bfdb3..65643a99c819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.execution.columnar +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.ByteOrder.nativeOrder + import scala.collection.JavaConverters._ import org.apache.commons.lang3.StringUtils @@ -30,6 +33,8 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.CollectionAccumulator @@ -53,17 +58,12 @@ private[sql] object InMemoryRelation { * @param stats The stat of columns */ private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], - dataTypes: Array[DataType], stats: InternalRow) { - def column(ordinal: Int): org.apache.spark.sql.execution.vectorized.ColumnVector = { - val dt = dataTypes(ordinal) +case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) { + def column(columnarIterator: ColumnarIterator, index: Int): ColumnVector = { + val ordinal = columnarIterator.getColumnIndexes(index) + val dataType = columnarIterator.getColumnTypes(index) val buffer = ByteBuffer.wrap(buffers(ordinal)).order(nativeOrder) - val accessor: BasicColumnAccessor[_] = dt match { - case BooleanType => new BooleanColumnAccessor(buffer) - case ByteType => new ByteColumnAccessor(buffer) - case ShortType => new ShortColumnAccessor(buffer) - case IntegerType | DateType => new IntColumnAccessor(buffer) - case LongType | TimestampType => new LongColumnAccessor(buffer) + val accessor: BasicColumnAccessor[_] = dataType match { case FloatType => new FloatColumnAccessor(buffer) case DoubleType => new DoubleColumnAccessor(buffer) } @@ -78,8 +78,7 @@ case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], (buffer, nullsBuffer) } - org.apache.spark.sql.execution.vectorized.ColumnVector.allocate(numRows, dt, - true, out, nullsBuffer) + ColumnVector.allocate(numRows, dataType, true, out, nullsBuffer) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala index 6f36707359ea..fd7e891d4d9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataFrameCacheSuite.scala @@ -92,7 +92,8 @@ class DataFrameCacheSuite extends QueryTest with SharedSQLContext { } test("Sort should be included in WholeStageCodegen without column codegen") { - val df = sqlContext.range(3, 0, -1).map(i => i.toFloat).toDF().sort(col("value")) + val df = sparkContext.parallelize(Seq(3.toFloat, 2.toFloat, 1.toFloat), 1).toDF() + .sort(col("value")) val plan = df.queryExecution.executedPlan assert(df.collect() === Array(Row(1.0), Row(2.0), Row(3.0))) assert(plan.find(p => From d035c42db42b6ecbc252b6972419451aabd6e06d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 8 Jul 2016 01:30:50 +0900 Subject: [PATCH 47/50] update benchmark --- .../spark/sql/DataFrameCacheBenchmark.scala | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala index d09abd4781db..61e8a9ec496f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala @@ -29,11 +29,14 @@ import org.apache.spark.util.Benchmark /** * Benchmark to measure performance of columnar storage for dataframe cache. * To run this: - * spark-submit --class + * bin/spark-submit --class org.apache.spark.sql.DataFrameCacheBenchmark + * sql/core/target/spark-sql_*-tests.jar + * [float datasize scale] [double datasize scale] [master URL] */ -object DataFrameCacheBenchmark { +case class DataFrameCacheBenchmark(masterURL: String) { val conf = new SparkConf() - val sc = new SparkContext("local[1]", "test-sql-context", conf) + val sc = new SparkContext( + (if (masterURL == null) "local[1]" else masterURL), "test-sql-context", conf) val sqlContext = new SQLContext(sc) import sqlContext.implicits._ @@ -114,9 +117,19 @@ object DataFrameCacheBenchmark { System.gc() } - def main(args: Array[String]): Unit = { - doubleSumBenchmark(1024 * 1024 * 15) - floatSumBenchmark(1024 * 1024 * 30) + def run(f: Int, d: Int): Unit = { + floatSumBenchmark(1024 * 1024 * f) + doubleSumBenchmark(1024 * 1024 * d) } } +object DataFrameCacheBenchmark { + def main(args: Array[String]): Unit = { + val f = if (args.length > 0) args(0).toInt else 30 + val d = if (args.length > 1) args(1).toInt else 15 + val masterURL = if (args.length > 2) args(2) else "local[1]" + + val benchmark = DataFrameCacheBenchmark(masterURL) + benchmark.run(f, d) + } +} From 54aabefc1fe431520bf8a7814ddeae5ce58aaa90 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 8 Jul 2016 02:01:53 +0900 Subject: [PATCH 48/50] fixed scala type error --- .../org/apache/spark/sql/DataFrameCacheBenchmark.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala index 61e8a9ec496f..e4c49013e01f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala @@ -29,8 +29,8 @@ import org.apache.spark.util.Benchmark /** * Benchmark to measure performance of columnar storage for dataframe cache. * To run this: - * bin/spark-submit --class org.apache.spark.sql.DataFrameCacheBenchmark - * sql/core/target/spark-sql_*-tests.jar + * bin/spark-submit --class org.apache.spark.sql.DataFrameCacheBenchmark + * sql/core/target/spark-sql_*-tests.jar * [float datasize scale] [double datasize scale] [master URL] */ case class DataFrameCacheBenchmark(masterURL: String) { @@ -117,7 +117,7 @@ case class DataFrameCacheBenchmark(masterURL: String) { System.gc() } - def run(f: Int, d: Int): Unit = { + def run(f: Int, d: Int): Unit = { floatSumBenchmark(1024 * 1024 * f) doubleSumBenchmark(1024 * 1024 * d) } From 4775db2ce9863f97dfc4dfe83fab823cf4730ada Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 9 Jul 2016 18:54:05 +0900 Subject: [PATCH 49/50] remove unnecessary final attribute --- .../vectorized/ByteBufferColumnVector.java | 90 +++++++++---------- 1 file changed, 45 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java index 5d35179c96d2..0392384007c6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ByteBufferColumnVector.java @@ -71,16 +71,16 @@ protected ByteBufferColumnVector(int capacity, DataType type, } @Override - public final long valuesNativeAddress() { + public long valuesNativeAddress() { throw new RuntimeException("Cannot get native address for on heap column"); } @Override - public final long nullsNativeAddress() { + public long nullsNativeAddress() { throw new RuntimeException("Cannot get native address for on heap column"); } @Override - public final void close() { + public void close() { } // @@ -88,19 +88,19 @@ public final void close() { // @Override - public final void putNotNull(int rowId) { + public void putNotNull(int rowId) { nulls[rowId] = (byte)0; } @Override - public final void putNull(int rowId) { + public void putNull(int rowId) { nulls[rowId] = (byte)1; ++numNulls; anyNullsSet = true; } @Override - public final void putNulls(int rowId, int count) { + public void putNulls(int rowId, int count) { for (int i = 0; i < count; ++i) { nulls[rowId + i] = (byte)1; } @@ -109,7 +109,7 @@ public final void putNulls(int rowId, int count) { } @Override - public final void putNotNulls(int rowId, int count) { + public void putNotNulls(int rowId, int count) { if (!anyNullsSet) return; for (int i = 0; i < count; ++i) { nulls[rowId + i] = (byte)0; @@ -117,7 +117,7 @@ public final void putNotNulls(int rowId, int count) { } @Override - public final boolean isNullAt(int rowId) { + public boolean isNullAt(int rowId) { return nulls[rowId] == 1; } @@ -126,17 +126,17 @@ public final boolean isNullAt(int rowId) { // @Override - public final void putBoolean(int rowId, boolean value) { + public void putBoolean(int rowId, boolean value) { throw new NotImplementedException(); } @Override - public final void putBooleans(int rowId, int count, boolean value) { + public void putBooleans(int rowId, int count, boolean value) { throw new NotImplementedException(); } @Override - public final boolean getBoolean(int rowId) { + public boolean getBoolean(int rowId) { throw new NotImplementedException(); } @@ -145,22 +145,22 @@ public final boolean getBoolean(int rowId) { // @Override - public final void putByte(int rowId, byte value) { + public void putByte(int rowId, byte value) { throw new NotImplementedException(); } @Override - public final void putBytes(int rowId, int count, byte value) { + public void putBytes(int rowId, int count, byte value) { throw new NotImplementedException(); } @Override - public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { + public void putBytes(int rowId, int count, byte[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final byte getByte(int rowId) { + public byte getByte(int rowId) { throw new NotImplementedException(); } @@ -169,22 +169,22 @@ public final byte getByte(int rowId) { // @Override - public final void putShort(int rowId, short value) { + public void putShort(int rowId, short value) { throw new NotImplementedException(); } @Override - public final void putShorts(int rowId, int count, short value) { + public void putShorts(int rowId, int count, short value) { throw new NotImplementedException(); } @Override - public final void putShorts(int rowId, int count, short[] src, int srcIndex) { + public void putShorts(int rowId, int count, short[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final short getShort(int rowId) { + public short getShort(int rowId) { throw new NotImplementedException(); } @@ -194,27 +194,27 @@ public final short getShort(int rowId) { // @Override - public final void putInt(int rowId, int value) { + public void putInt(int rowId, int value) { throw new NotImplementedException(); } @Override - public final void putInts(int rowId, int count, int value) { + public void putInts(int rowId, int count, int value) { throw new NotImplementedException(); } @Override - public final void putInts(int rowId, int count, int[] src, int srcIndex) { + public void putInts(int rowId, int count, int[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final int getInt(int rowId) { + public int getInt(int rowId) { throw new NotImplementedException(); } @@ -223,27 +223,27 @@ public final int getInt(int rowId) { // @Override - public final void putLong(int rowId, long value) { + public void putLong(int rowId, long value) { throw new NotImplementedException(); } @Override - public final void putLongs(int rowId, int count, long value) { + public void putLongs(int rowId, int count, long value) { throw new NotImplementedException(); } @Override - public final void putLongs(int rowId, int count, long[] src, int srcIndex) { + public void putLongs(int rowId, int count, long[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final long getLong(int rowId) { + public long getLong(int rowId) { throw new NotImplementedException(); } @@ -252,27 +252,27 @@ public final long getLong(int rowId) { // @Override - public final void putFloat(int rowId, float value) { + public void putFloat(int rowId, float value) { throw new NotImplementedException(); } @Override - public final void putFloats(int rowId, int count, float value) { + public void putFloats(int rowId, int count, float value) { throw new NotImplementedException(); } @Override - public final void putFloats(int rowId, int count, float[] src, int srcIndex) { + public void putFloats(int rowId, int count, float[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { + public void putFloats(int rowId, int count, byte[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final float getFloat(int rowId) { + public float getFloat(int rowId) { assert(dictionary == null); return Platform.getFloat(data, offset + rowId * 4); } @@ -282,27 +282,27 @@ public final float getFloat(int rowId) { // @Override - public final void putDouble(int rowId, double value) { + public void putDouble(int rowId, double value) { throw new NotImplementedException(); } @Override - public final void putDoubles(int rowId, int count, double value) { + public void putDoubles(int rowId, int count, double value) { throw new NotImplementedException(); } @Override - public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + public void putDoubles(int rowId, int count, double[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { throw new NotImplementedException(); } @Override - public final double getDouble(int rowId) { + public double getDouble(int rowId) { assert(dictionary == null); return Platform.getDouble(data, offset + rowId * 8); } @@ -312,17 +312,17 @@ public final double getDouble(int rowId) { // @Override - public final int getArrayLength(int rowId) { throw new NotImplementedException(); } + public int getArrayLength(int rowId) { throw new NotImplementedException(); } @Override - public final int getArrayOffset(int rowId) { throw new NotImplementedException(); } + public int getArrayOffset(int rowId) { throw new NotImplementedException(); } @Override - public final void putArray(int rowId, int offset, int length) { + public void putArray(int rowId, int offset, int length) { throw new NotImplementedException(); } @Override - public final void loadBytes(ColumnVector.Array array) { + public void loadBytes(ColumnVector.Array array) { throw new NotImplementedException(); } @@ -331,12 +331,12 @@ public final void loadBytes(ColumnVector.Array array) { // @Override - public final int putByteArray(int rowId, byte[] value, int offset, int length) { + public int putByteArray(int rowId, byte[] value, int offset, int length) { throw new NotImplementedException(); } @Override - public final void reserve(int requiredCapacity) { + public void reserve(int requiredCapacity) { if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); } From fea9f3413b66df6459eb9a0e2cc9359e12dd08ab Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 9 Jul 2016 18:54:18 +0900 Subject: [PATCH 50/50] update --- .../spark/sql/DataFrameCacheBenchmark.scala | 50 ++++++++++--------- 1 file changed, 27 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala index e4c49013e01f..73631a35460e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameCacheBenchmark.scala @@ -33,17 +33,9 @@ import org.apache.spark.util.Benchmark * sql/core/target/spark-sql_*-tests.jar * [float datasize scale] [double datasize scale] [master URL] */ -case class DataFrameCacheBenchmark(masterURL: String) { - val conf = new SparkConf() - val sc = new SparkContext( - (if (masterURL == null) "local[1]" else masterURL), "test-sql-context", conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ +class DataFrameCacheBenchmark { - // Set default configs. Individual cases will change them if necessary. - sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - - def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + def withSQLConf(sqlContext: SQLContext, pairs: (String, String)*)(f: => Unit): Unit = { val (keys, values) = pairs.unzip val currentValues = keys.map(key => Try(sqlContext.conf.getConfString(key)).toOption) (keys, values).zipped.foreach(sqlContext.conf.setConfString) @@ -55,18 +47,20 @@ case class DataFrameCacheBenchmark(masterURL: String) { } } - def floatSumBenchmark(values: Int, iters: Int = 5): Unit = { + def floatSumBenchmark(sqlContext: SQLContext, values: Int, iters: Int = 5): Unit = { + import sqlContext.implicits._ + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) val benchmarkPT = new Benchmark("Float Sum with PassThrough cache", values, iters) val rand1 = new Random(511) - val dfPassThrough = sc.parallelize(0 to values - 1, 1) + val dfPassThrough = sqlContext.sparkContext.parallelize(0 to values - 1, 1) .map(i => rand1.nextFloat()).toDF().cache() dfPassThrough.count() // force to create df.cache() suites.foreach { case (str, value) => benchmarkPT.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + withSQLConf(sqlContext, SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { dfPassThrough.agg(sum("value")).collect } } @@ -86,18 +80,20 @@ case class DataFrameCacheBenchmark(masterURL: String) { System.gc() } - def doubleSumBenchmark(values: Int, iters: Int = 5): Unit = { + def doubleSumBenchmark(sqlContext: SQLContext, values: Int, iters: Int = 5): Unit = { + import sqlContext.implicits._ + val suites = Seq(("InternalRow", "false"), ("ColumnVector", "true")) val benchmarkPT = new Benchmark("Double Sum with PassThrough cache", values, iters) val rand1 = new Random(511) - val dfPassThrough = sc.parallelize(0 to values - 1, 1) + val dfPassThrough = sqlContext.sparkContext.parallelize(0 to values - 1, 1) .map(i => rand1.nextDouble()).toDF().cache() dfPassThrough.count() // force to create df.cache() suites.foreach { case (str, value) => benchmarkPT.addCase(s"$str codegen") { iter => - withSQLConf(SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { + withSQLConf(sqlContext, SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { dfPassThrough.agg(sum("value")).collect } } @@ -117,19 +113,27 @@ case class DataFrameCacheBenchmark(masterURL: String) { System.gc() } - def run(f: Int, d: Int): Unit = { - floatSumBenchmark(1024 * 1024 * f) - doubleSumBenchmark(1024 * 1024 * d) + def run(sqlContext: SQLContext, f: Int, d: Int): Unit = { + sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + + floatSumBenchmark(sqlContext, 1024 * 1024 * f) + doubleSumBenchmark(sqlContext, 1024 * 1024 * d) } } object DataFrameCacheBenchmark { + val F = 30 + val D = 15 def main(args: Array[String]): Unit = { - val f = if (args.length > 0) args(0).toInt else 30 - val d = if (args.length > 1) args(1).toInt else 15 + val f = if (args.length > 0) args(0).toInt else F + val d = if (args.length > 1) args(1).toInt else D val masterURL = if (args.length > 2) args(2) else "local[1]" - val benchmark = DataFrameCacheBenchmark(masterURL) - benchmark.run(f, d) + val conf = new SparkConf() + val sc = new SparkContext(masterURL, "DataFrameCacheBenchmark", conf) + val sqlContext = new SQLContext(sc) + + val benchmark = new DataFrameCacheBenchmark + benchmark.run(sqlContext, f, d) } }