From 8ed3ee78fcfc03ca75e409c7e94bad6bb446a04a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 17 Jun 2016 14:47:20 -0700 Subject: [PATCH 01/40] Add benchmark code --- .../execution/columnar/InMemoryRelation.scala | 45 +++++++++++++++++++ .../benchmark/AggregateBenchmark.scala | 20 ++++++--- 2 files changed, 59 insertions(+), 6 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 37bd95e73778..c096ac4b7123 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils +import org.apache.spark.memory.MemoryMode import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystConf, InternalRow} @@ -27,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ 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.storage.StorageLevel import org.apache.spark.util.LongAccumulator @@ -145,6 +147,49 @@ case class InMemoryRelation( _cachedColumnBuffers = cached } +// private def buildBuffers(): Unit = { +// val output = child.output +// val cached = child.execute().mapPartitionsInternal { rowIterator => +// new Iterator[ColumnarCachedBatch] { +// def next(): ColumnarCachedBatch = { +// val columnVectors = output.map { attribute => +// ColumnVector.allocate(batchSize, attribute.dataType, MemoryMode.ON_HEAP) +// }.toArray +// +// var rowCount = 0 +// var totalSize = 0L +// while (rowIterator.hasNext && rowCount < batchSize +// && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { +// val row = rowIterator.next() +// assert( +// row.numFields == columnVectors.length, +// s"Row column number mismatch, expected ${output.size} columns, " + +// s"but got ${row.numFields}." + +// s"\nRow content: $row") +// +// var i = 0 +// totalSize = 0 +// while (i < row.numFields) { +// columnVectors(i).putLong(rowCount, row.getLong(i)) +// totalSize += 8 +// i += 1 +// } +// rowCount += 1 +// } +// +// ColumnarCachedBatch(rowCount, columnVectors) +// } +// +// def hasNext: Boolean = rowIterator.hasNext +// } +// }.persist(storageLevel) +// +// cached.setName( +// tableName.map(n => s"In-memory table $n") +// .getOrElse(StringUtils.abbreviate(child.toString, 1024))) +// _cachedColumnBuffers = cached +// } + def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 8a2993bdf4b2..11ed9e97e3d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -131,18 +131,26 @@ class AggregateBenchmark extends BenchmarkBase { */ } - ignore("aggregate with randomized keys") { + + test("aggregate with randomized keys") { val N = 20 << 22 + val numIters = 10 val benchmark = new Benchmark("Aggregate w keys", N) sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() - - benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) - f() + def run(cache: Boolean = false): Unit = { + if (cache) { + sparkSession.catalog.cacheTable("test") + } + try { + val ds = sparkSession.sql("select k, sum(id) from test group by k") + ds.collect() + ds.collect() + } finally { + sparkSession.catalog.clearCache() + } } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => From 6319cd48c345b7a70cdaa129bbcc87a9048fc899 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 20 Jun 2016 11:35:21 -0700 Subject: [PATCH 02/40] backup --- .../execution/vectorized/ColumnVector.java | 11 ++- .../vectorized/OnHeapColumnVector.java | 26 ++++++ .../execution/columnar/InMemoryRelation.scala | 88 ++++++++++--------- .../columnar/InMemoryTableScanExec.scala | 12 +-- 4 files changed, 85 insertions(+), 52 deletions(-) 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 354c878aca00..76e92d41b0ca 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 @@ -16,6 +16,9 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.io.Externalizable; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.math.BigDecimal; import java.math.BigInteger; @@ -57,7 +60,11 @@ * * ColumnVectors are intended to be reused. */ -public abstract class ColumnVector implements AutoCloseable { +public abstract class ColumnVector implements AutoCloseable {// , Externalizable { + +// public void writeExternal(ObjectOutput out) throws java.io.IOException { } +// public void readExternal(ObjectInput in) throws java.io.IOException { } + /** * Allocates a column to store elements of `type` on or off heap. * Capacity is the initial capacity of the vector and it will grow as necessary. Capacity is @@ -897,7 +904,7 @@ public final int appendStruct(boolean isNull) { /** * Data type for this column. */ - protected final DataType type; + protected DataType type; /** * Number of nulls in this column. This is an optimization for the reader, to skip NULL checks. diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index 9b410bacff5d..6c22cc6020fd 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; @@ -30,6 +32,30 @@ */ public final class OnHeapColumnVector extends ColumnVector { +// // capacity, num longs, long1, long2, long3... +// public void writeExternal(ObjectOutput out) throws java.io.IOException { +// out.writeInt(capacity); +// out.writeInt(longData.length); +// int i; +// for (i = 0; i < longData.length; i++) { +// out.writeLong(longData[i]); +// } +// } +// +// public void readExternal(ObjectInput in) throws java.io.IOException { +// capacity = in.readInt(); +// int numLongs = in.readInt(); +// longData = new long[numLongs]; +// int i; +// for (i = 0; i < numLongs; i++) { +// longData[i] = in.readLong(); +// } +// } +// +// public OnHeapColumnVector() { +// super(10000, LongType$.MODULE$, MemoryMode.ON_HEAP); +// } + private static final boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); 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 c096ac4b7123..4ba835efc69b 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 @@ -85,12 +85,14 @@ case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { buildBuffers() + buildBuffers2() } def recache(): Unit = { _cachedColumnBuffers.unpersist() _cachedColumnBuffers = null buildBuffers() + buildBuffers2() } private def buildBuffers(): Unit = { @@ -147,48 +149,49 @@ case class InMemoryRelation( _cachedColumnBuffers = cached } -// private def buildBuffers(): Unit = { -// val output = child.output -// val cached = child.execute().mapPartitionsInternal { rowIterator => -// new Iterator[ColumnarCachedBatch] { -// def next(): ColumnarCachedBatch = { -// val columnVectors = output.map { attribute => -// ColumnVector.allocate(batchSize, attribute.dataType, MemoryMode.ON_HEAP) -// }.toArray -// -// var rowCount = 0 -// var totalSize = 0L -// while (rowIterator.hasNext && rowCount < batchSize -// && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { -// val row = rowIterator.next() -// assert( -// row.numFields == columnVectors.length, -// s"Row column number mismatch, expected ${output.size} columns, " + -// s"but got ${row.numFields}." + -// s"\nRow content: $row") -// -// var i = 0 -// totalSize = 0 -// while (i < row.numFields) { -// columnVectors(i).putLong(rowCount, row.getLong(i)) -// totalSize += 8 -// i += 1 -// } -// rowCount += 1 -// } -// -// ColumnarCachedBatch(rowCount, columnVectors) -// } -// -// def hasNext: Boolean = rowIterator.hasNext -// } -// }.persist(storageLevel) -// -// cached.setName( -// tableName.map(n => s"In-memory table $n") -// .getOrElse(StringUtils.abbreviate(child.toString, 1024))) -// _cachedColumnBuffers = cached -// } + private def buildBuffers2(): Unit = { + val output = child.output + val cached = child.execute().mapPartitionsInternal { rowIterator => + new Iterator[ColumnarCachedBatch] { + def next(): ColumnarCachedBatch = { + val columnVectors = output.map { attribute => + new Array[Long](batchSize) + // ColumnVector.allocate(batchSize, attribute.dataType, MemoryMode.ON_HEAP) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext && rowCount < batchSize + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + val row = rowIterator.next() + assert( + row.numFields == columnVectors.length, + s"Row column number mismatch, expected ${output.size} columns, " + + s"but got ${row.numFields}." + + s"\nRow content: $row") + + var i = 0 + totalSize = 0 + while (i < row.numFields) { + columnVectors(i)(rowCount) = row.getLong(i) + totalSize += 8 + i += 1 + } + rowCount += 1 + } + + ColumnarCachedBatch(rowCount, columnVectors) + } + + def hasNext: Boolean = rowIterator.hasNext + } + }.persist(storageLevel) + + cached.setName( + tableName.map(n => s"In-memory table $n") + .getOrElse(StringUtils.abbreviate(child.toString, 1024))) + _cachedColumnVectors = cached + } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( @@ -209,6 +212,7 @@ case class InMemoryRelation( } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers + def cachedColumnVectors: RDD[ColumnarCachedBatch] = _cachedColumnVectors override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, batchStats) 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 9028caa446e8..ff95369f8601 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 @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.rdd.RDD 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.expressions.codegen.{BufferHolder, UnsafeRowWriter} 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 case class InMemoryTableScanExec( @@ -115,8 +113,6 @@ case class InMemoryTableScanExec( lazy val readPartitions = sparkContext.longAccumulator lazy val readBatches = sparkContext.longAccumulator - private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning - protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") @@ -159,9 +155,10 @@ case class InMemoryTableScanExec( } else { true } + hasNext + } else { + true // currentBatch != null } - } else { - cachedBatchIterator } // update SQL metrics @@ -182,7 +179,6 @@ case class InMemoryTableScanExec( if (enableAccumulators && columnarIterator.hasNext) { readPartitions.add(1) } - columnarIterator } } } From bf5a1f93b2fc8fae312925759e7293f8b8c6afd9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 20 Jun 2016 15:36:44 -0700 Subject: [PATCH 03/40] Narrow benchmarked code + add back old scan code --- .../org/apache/spark/util/Benchmark.scala | 17 +++-- .../vectorized/OnHeapColumnVector.java | 44 ++++++------ .../execution/columnar/InMemoryRelation.scala | 25 +++---- .../columnar/InMemoryTableScanExec.scala | 67 +++++++++++++++++-- .../benchmark/AggregateBenchmark.scala | 41 +++++------- 5 files changed, 126 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 7def44bd2a2b..4346cb2d175e 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -69,11 +69,20 @@ private[spark] class Benchmark( * @param name of the benchmark case * @param numIters if non-zero, forces exactly this many iterations to be run */ - def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { + def addCase( + name: String, + numIters: Int = 0, + prepare: () => Unit = () => { }, + cleanup: () => Unit = () => { })(f: Int => Unit): Unit = { addTimerCase(name, numIters) { timer => - timer.startTiming() - f(timer.iteration) - timer.stopTiming() + try { + prepare() + timer.startTiming() + f(timer.iteration) + } finally { + timer.stopTiming() + cleanup() + } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index 6c22cc6020fd..fe5857ff5950 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -33,28 +33,28 @@ public final class OnHeapColumnVector extends ColumnVector { // // capacity, num longs, long1, long2, long3... -// public void writeExternal(ObjectOutput out) throws java.io.IOException { -// out.writeInt(capacity); -// out.writeInt(longData.length); -// int i; -// for (i = 0; i < longData.length; i++) { -// out.writeLong(longData[i]); -// } -// } -// -// public void readExternal(ObjectInput in) throws java.io.IOException { -// capacity = in.readInt(); -// int numLongs = in.readInt(); -// longData = new long[numLongs]; -// int i; -// for (i = 0; i < numLongs; i++) { -// longData[i] = in.readLong(); -// } -// } -// -// public OnHeapColumnVector() { -// super(10000, LongType$.MODULE$, MemoryMode.ON_HEAP); -// } + public void writeExternal(ObjectOutput out) throws java.io.IOException { + out.writeInt(capacity); + out.writeInt(longData.length); + int i; + for (i = 0; i < longData.length; i++) { + out.writeLong(longData[i]); + } + } + + public void readExternal(ObjectInput in) throws java.io.IOException { + capacity = in.readInt(); + int numLongs = in.readInt(); + longData = new long[numLongs]; + int i; + for (i = 0; i < numLongs; i++) { + longData[i] = in.readLong(); + } + } + + public OnHeapColumnVector() { + super(10000, LongType$.MODULE$, MemoryMode.ON_HEAP); + } private static final boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); 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 4ba835efc69b..6501f71861d4 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 @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.expressions._ 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.execution.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel import org.apache.spark.util.LongAccumulator @@ -149,38 +150,32 @@ case class InMemoryRelation( _cachedColumnBuffers = cached } + // IWASHERE private def buildBuffers2(): Unit = { - val output = child.output + val schema = StructType.fromAttributes(child.output) val cached = child.execute().mapPartitionsInternal { rowIterator => new Iterator[ColumnarCachedBatch] { def next(): ColumnarCachedBatch = { - val columnVectors = output.map { attribute => - new Array[Long](batchSize) - // ColumnVector.allocate(batchSize, attribute.dataType, MemoryMode.ON_HEAP) - }.toArray - + val columnarBatch = ColumnarBatch.allocate(schema, MemoryMode.ON_HEAP) var rowCount = 0 var totalSize = 0L while (rowIterator.hasNext && rowCount < batchSize && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { val row = rowIterator.next() - assert( - row.numFields == columnVectors.length, - s"Row column number mismatch, expected ${output.size} columns, " + - s"but got ${row.numFields}." + - s"\nRow content: $row") - + assert(row.numFields == columnarBatch.numCols, "Row column number mismatch, " + + s"expected ${columnarBatch.numCols} columns, but got ${row.numFields}. \n" + + s"Row content: $row") var i = 0 totalSize = 0 while (i < row.numFields) { - columnVectors(i)(rowCount) = row.getLong(i) + columnarBatch.column(i).putLong(rowCount, row.getLong(i)) totalSize += 8 i += 1 } rowCount += 1 } - ColumnarCachedBatch(rowCount, columnVectors) + ColumnarCachedBatch(rowCount, columnarBatch) } def hasNext: Boolean = rowIterator.hasNext 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 ff95369f8601..dd0905eb327f 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 @@ -19,11 +19,13 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.rdd.RDD 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.expressions.codegen.{BufferHolder, UnsafeRowWriter} 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 case class InMemoryTableScanExec( @@ -113,6 +115,8 @@ case class InMemoryTableScanExec( lazy val readPartitions = sparkContext.longAccumulator lazy val readBatches = sparkContext.longAccumulator + private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning + protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") @@ -155,10 +159,9 @@ case class InMemoryTableScanExec( } else { true } - hasNext - } else { - true // currentBatch != null } + } else { + cachedBatchIterator } // update SQL metrics @@ -179,6 +182,62 @@ case class InMemoryTableScanExec( if (enableAccumulators && columnarIterator.hasNext) { readPartitions.add(1) } + + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) + if (enableAccumulators && columnarIterator.hasNext) { + readPartitions.add(1) + } + columnarIterator } } + +// protected override def doExecute(): RDD[InternalRow] = { +// val childOutput = relation.child.output +// relation.cachedColumnVectors.mapPartitionsInternal { batchIter => +// new Iterator[InternalRow] { +// private val unsafeRow = new UnsafeRow(childOutput.size) +// private val bufferHolder = new BufferHolder(unsafeRow) +// private val rowWriter = new UnsafeRowWriter(bufferHolder, childOutput.size) +// private var currentBatch: ColumnarCachedBatch = null +// private var currentRowIndex = 0 // row index within each batch +// +// override def hasNext: Boolean = { +// if (currentBatch == null) { +// val hasNext = batchIter.hasNext +// if (hasNext) { +// currentBatch = batchIter.next() +// currentRowIndex = 0 +// } +// hasNext +// } else { +// true // currentBatch != null +// } +// } +// +// override def next(): InternalRow = { +// if (currentBatch == null) { +// throw new NoSuchElementException +// } +// rowWriter.zeroOutNullBytes() +// // Populate the row +// childOutput.zipWithIndex.foreach { case (attr, colIndex) => +// val colValue = currentBatch.buffers(colIndex)(currentRowIndex) +// rowWriter.write(colIndex, colValue) +// } +// // If we have consumed this batch, move onto the next one +// currentRowIndex += 1 +// if (currentRowIndex == currentBatch.numRows) { +// currentBatch = null +// } +// unsafeRow +// } +// } +// } +// } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 11ed9e97e3d6..5b74eac68405 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -132,25 +132,31 @@ class AggregateBenchmark extends BenchmarkBase { } - test("aggregate with randomized keys") { - val N = 20 << 22 + test("cache aggregate with randomized keys") { + val N = 20 << 21 val numIters = 10 - val benchmark = new Benchmark("Aggregate w keys", N) - sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + val benchmark = new Benchmark("Cache aggregate", N) + sparkSession.range(N) + .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - def run(cache: Boolean = false): Unit = { - if (cache) { - sparkSession.catalog.cacheTable("test") + /** + * Actually run the benchmark, optionally specifying whether to cache the dataset. + */ + def runBenchmark(name: String, cache: Boolean, params: Map[String, String]): Unit = { + val ds = sparkSession.sql("select k, sum(id) from test group by k") + val defaults = params.keys.map { k => (k, sparkSession.conf.get(k)) } + val prepare = () => { + params.foreach { case (k, v) => sparkSession.conf.set(k, v) } + if (cache) { sparkSession.catalog.cacheTable("test") } + ds.collect(): Unit } - try { - val ds = sparkSession.sql("select k, sum(id) from test group by k") - ds.collect() - ds.collect() - } finally { + val cleanup = () => { + defaults.foreach { case (k, v) => sparkSession.conf.set(k, v) } sparkSession.catalog.clearCache() } + benchmark.addCase(name, numIters, prepare, cleanup) { _ => ds.collect() } } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => @@ -167,17 +173,6 @@ class AggregateBenchmark extends BenchmarkBase { } benchmark.run() - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - codegen = F 7445 / 7517 11.3 88.7 1.0X - codegen = T hashmap = F 4672 / 4703 18.0 55.7 1.6X - codegen = T hashmap = T 1764 / 1958 47.6 21.0 4.2X - */ } ignore("aggregate with string key") { From ba2f329c096e153a2c3b85d4a109b18455ff33bc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Jun 2016 11:28:19 -0700 Subject: [PATCH 04/40] Fix benchmark to time only the read path --- .../spark/scheduler/TaskSetManager.scala | 2 +- .../org/apache/spark/util/Benchmark.scala | 28 +++++++++++-------- .../benchmark/AggregateBenchmark.scala | 21 ++++++++++---- .../execution/benchmark/BenchmarkBase.scala | 4 ++- 4 files changed, 37 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c7ff13cebfaa..af6d86908cd3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -738,7 +738,7 @@ private[spark] class TaskSetManager( s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { case fetchFailed: FetchFailed => - logWarning(failureReason) + // logWarning(failureReason) if (!successful(index)) { successful(index) = true tasksSuccessful += 1 diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 4346cb2d175e..7576faa99c96 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -74,16 +74,12 @@ private[spark] class Benchmark( numIters: Int = 0, prepare: () => Unit = () => { }, cleanup: () => Unit = () => { })(f: Int => Unit): Unit = { - addTimerCase(name, numIters) { timer => - try { - prepare() - timer.startTiming() - f(timer.iteration) - } finally { - timer.stopTiming() - cleanup() - } + val timedF = (timer: Benchmark.Timer) => { + timer.startTiming() + f(timer.iteration) + timer.stopTiming() } + benchmarks += Benchmark.Case(name, timedF, numIters, prepare, cleanup) } /** @@ -110,7 +106,12 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - measure(valuesPerIteration, c.numIters)(c.fn) + try { + c.prepare() + measure(valuesPerIteration, c.numIters)(c.fn) + } finally { + c.cleanup() + } } println @@ -197,7 +198,12 @@ private[spark] object Benchmark { } } - case class Case(name: String, fn: Timer => Unit, numIters: Int) + case class Case( + name: String, + fn: Timer => Unit, + numIters: Int, + prepare: () => Unit = () => { }, + cleanup: () => Unit = () => { }) case class Result(avgMs: Double, bestRate: Double, bestMs: Double) /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 5b74eac68405..df2feeddeb5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -133,30 +133,41 @@ class AggregateBenchmark extends BenchmarkBase { test("cache aggregate with randomized keys") { - val N = 20 << 21 - + val N = 20 << 20 val numIters = 10 val benchmark = new Benchmark("Cache aggregate", N) sparkSession.range(N) .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") + /** + * Call collect on the dataset after deleting all existing temporary files. + */ + def doCollect(ds: org.apache.spark.sql.Dataset[_]): Unit = { + ds.sparkSession.sparkContext.parallelize(1 to 10, 10).foreach { _ => + org.apache.spark.SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach { dir => + dir.delete() + } + } + ds.collect() + } + /** * Actually run the benchmark, optionally specifying whether to cache the dataset. */ - def runBenchmark(name: String, cache: Boolean, params: Map[String, String]): Unit = { + def addBenchmark(name: String, cache: Boolean, params: Map[String, String]): Unit = { val ds = sparkSession.sql("select k, sum(id) from test group by k") val defaults = params.keys.map { k => (k, sparkSession.conf.get(k)) } val prepare = () => { params.foreach { case (k, v) => sparkSession.conf.set(k, v) } if (cache) { sparkSession.catalog.cacheTable("test") } - ds.collect(): Unit + doCollect(ds) } val cleanup = () => { defaults.foreach { case (k, v) => sparkSession.conf.set(k, v) } sparkSession.catalog.clearCache() } - benchmark.addCase(name, numIters, prepare, cleanup) { _ => ds.collect() } + benchmark.addCase(name, numIters, prepare, cleanup) { _ => doCollect(ds) } } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala index c99a5aec1cd6..96445beec6fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala @@ -28,8 +28,10 @@ import org.apache.spark.util.Benchmark private[benchmark] trait BenchmarkBase extends SparkFunSuite { lazy val sparkSession = SparkSession.builder - .master("local[1]") + .master("local-cluster[2,1,1024]") .appName("microbenchmark") + .config("spark.ui.enabled", true) + .config("spark.ui.port", 5050) .config("spark.sql.shuffle.partitions", 1) .config("spark.sql.autoBroadcastJoinThreshold", 1) .getOrCreate() From b9d9346f7e82cc11a7821f96a8d211f7e46c0a7b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Jun 2016 14:35:01 -0700 Subject: [PATCH 05/40] First working impl. of ColumnarBatch based caching Note, this doesn't work: spark.table("tab1").collect(), because we're trying to cast ColumnarBatch.Row into UnsafeRow. This works, however: spark.table("tab1").groupBy("i").sum("j").collect(). --- .../execution/columnar/InMemoryRelation.scala | 16 ++-- .../columnar/InMemoryTableScanExec.scala | 81 +++++++++---------- .../benchmark/AggregateBenchmark.scala | 17 ++-- 3 files changed, 56 insertions(+), 58 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 6501f71861d4..0dd1cbc21918 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ 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.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.execution.vectorized.ColumnarBatch import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel import org.apache.spark.util.LongAccumulator @@ -154,9 +154,9 @@ case class InMemoryRelation( private def buildBuffers2(): Unit = { val schema = StructType.fromAttributes(child.output) val cached = child.execute().mapPartitionsInternal { rowIterator => - new Iterator[ColumnarCachedBatch] { - def next(): ColumnarCachedBatch = { - val columnarBatch = ColumnarBatch.allocate(schema, MemoryMode.ON_HEAP) + new Iterator[ColumnarBatch] { + def next(): ColumnarBatch = { + val columnarBatch = ColumnarBatch.allocate(schema, MemoryMode.ON_HEAP, batchSize) var rowCount = 0 var totalSize = 0L while (rowIterator.hasNext && rowCount < batchSize @@ -168,14 +168,14 @@ case class InMemoryRelation( var i = 0 totalSize = 0 while (i < row.numFields) { - columnarBatch.column(i).putLong(rowCount, row.getLong(i)) + columnarBatch.column(i).appendLong(row.getLong(i)) totalSize += 8 i += 1 } rowCount += 1 } - - ColumnarCachedBatch(rowCount, columnarBatch) + columnarBatch.setNumRows(rowCount) + columnarBatch } def hasNext: Boolean = rowIterator.hasNext @@ -207,7 +207,7 @@ case class InMemoryRelation( } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers - def cachedColumnVectors: RDD[ColumnarCachedBatch] = _cachedColumnVectors + def cachedColumnVectors: RDD[ColumnarBatch] = _cachedColumnVectors override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, batchStats) 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 dd0905eb327f..57ca6eadf246 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.ColumnarBatch import org.apache.spark.sql.types.UserDefinedType @@ -117,7 +118,7 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning - protected override def doExecute(): RDD[InternalRow] = { + private def doExecuteRow(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") if (enableAccumulators) { @@ -196,48 +197,40 @@ case class InMemoryTableScanExec( } } -// protected override def doExecute(): RDD[InternalRow] = { -// val childOutput = relation.child.output -// relation.cachedColumnVectors.mapPartitionsInternal { batchIter => -// new Iterator[InternalRow] { -// private val unsafeRow = new UnsafeRow(childOutput.size) -// private val bufferHolder = new BufferHolder(unsafeRow) -// private val rowWriter = new UnsafeRowWriter(bufferHolder, childOutput.size) -// private var currentBatch: ColumnarCachedBatch = null -// private var currentRowIndex = 0 // row index within each batch -// -// override def hasNext: Boolean = { -// if (currentBatch == null) { -// val hasNext = batchIter.hasNext -// if (hasNext) { -// currentBatch = batchIter.next() -// currentRowIndex = 0 -// } -// hasNext -// } else { -// true // currentBatch != null -// } -// } -// -// override def next(): InternalRow = { -// if (currentBatch == null) { -// throw new NoSuchElementException -// } -// rowWriter.zeroOutNullBytes() -// // Populate the row -// childOutput.zipWithIndex.foreach { case (attr, colIndex) => -// val colValue = currentBatch.buffers(colIndex)(currentRowIndex) -// rowWriter.write(colIndex, colValue) -// } -// // If we have consumed this batch, move onto the next one -// currentRowIndex += 1 -// if (currentRowIndex == currentBatch.numRows) { -// currentBatch = null -// } -// unsafeRow -// } -// } -// } -// } + private def doExecuteColumnar(): RDD[InternalRow] = { + relation.cachedColumnVectors.mapPartitionsInternal { batchIter => + new Iterator[InternalRow] { + private var currentRowIterator: java.util.Iterator[ColumnarBatch.Row] = null + + override def hasNext: Boolean = { + if (currentRowIterator == null && batchIter.hasNext) { + currentRowIterator = batchIter.next().rowIterator + } + currentRowIterator != null && currentRowIterator.hasNext + } + + override def next(): InternalRow = { + if (!hasNext) { + throw new NoSuchElementException + } + val row = currentRowIterator.next() + if (!currentRowIterator.hasNext) { + currentRowIterator = null + } + row + } + } + } + } + + protected override def doExecute(): RDD[InternalRow] = { + val useColumnarScan = relation.child.sqlContext.conf.getConfString( + "spark.sql.inMemoryColumnarScan", "true").toBoolean + if (useColumnarScan) { + doExecuteColumnar() + } else { + doExecuteRow() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index df2feeddeb5a..19175a274c8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -21,6 +21,7 @@ import java.util.HashMap import org.apache.spark.SparkConf import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap @@ -133,31 +134,35 @@ class AggregateBenchmark extends BenchmarkBase { test("cache aggregate with randomized keys") { - val N = 20 << 20 + val N = 20 << 19 val numIters = 10 val benchmark = new Benchmark("Cache aggregate", N) sparkSession.range(N) .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") + val expectedAnswer = sparkSession.sql("select k, sum(id) from test group by k").collect().toSeq /** * Call collect on the dataset after deleting all existing temporary files. */ - def doCollect(ds: org.apache.spark.sql.Dataset[_]): Unit = { - ds.sparkSession.sparkContext.parallelize(1 to 10, 10).foreach { _ => + def doCollect(df: org.apache.spark.sql.DataFrame): Unit = { + df.sparkSession.sparkContext.parallelize(1 to 10, 10).foreach { _ => org.apache.spark.SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach { dir => dir.delete() } } - ds.collect() + QueryTest.checkAnswer(df, expectedAnswer) match { + case Some(errMessage) => throw new RuntimeException(errMessage) + case None => // all good + } } /** * Actually run the benchmark, optionally specifying whether to cache the dataset. */ - def addBenchmark(name: String, cache: Boolean, params: Map[String, String]): Unit = { + def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { val ds = sparkSession.sql("select k, sum(id) from test group by k") - val defaults = params.keys.map { k => (k, sparkSession.conf.get(k)) } + val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } val prepare = () => { params.foreach { case (k, v) => sparkSession.conf.set(k, v) } if (cache) { sparkSession.catalog.cacheTable("test") } From 23a50d968929a013f469df7a92b7ce96ea32c4d7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Jun 2016 17:45:34 -0700 Subject: [PATCH 06/40] Always enable codegen and vectorized hashmap --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 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 0dd1cbc21918..a83217dcb256 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 @@ -86,14 +86,14 @@ case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { buildBuffers() - buildBuffers2() + buildColumnarBuffers() } def recache(): Unit = { _cachedColumnBuffers.unpersist() _cachedColumnBuffers = null buildBuffers() - buildBuffers2() + buildColumnarBuffers() } private def buildBuffers(): Unit = { @@ -150,8 +150,7 @@ case class InMemoryRelation( _cachedColumnBuffers = cached } - // IWASHERE - private def buildBuffers2(): Unit = { + private def buildColumnarBuffers(): Unit = { val schema = StructType.fromAttributes(child.output) val cached = child.execute().mapPartitionsInternal { rowIterator => new Iterator[ColumnarBatch] { @@ -168,7 +167,7 @@ case class InMemoryRelation( var i = 0 totalSize = 0 while (i < row.numFields) { - columnarBatch.column(i).appendLong(row.getLong(i)) + columnarBatch.column(i).putLong(rowCount, row.getLong(i)) totalSize += 8 i += 1 } From b25841d2b49954adff46394942ad84f31f0ad722 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 12:48:37 -0700 Subject: [PATCH 07/40] Don't benchmark aggregate --- .../sql/execution/benchmark/AggregateBenchmark.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 19175a274c8f..142b96b5499a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -133,14 +133,14 @@ class AggregateBenchmark extends BenchmarkBase { } - test("cache aggregate with randomized keys") { - val N = 20 << 19 + test("cache with randomized keys") { + val N = 20 << 20 val numIters = 10 val benchmark = new Benchmark("Cache aggregate", N) sparkSession.range(N) .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - val expectedAnswer = sparkSession.sql("select k, sum(id) from test group by k").collect().toSeq + val expectedAnswer = sparkSession.sql("select count(k), count(id) from test").collect().toSeq /** * Call collect on the dataset after deleting all existing temporary files. @@ -161,7 +161,7 @@ class AggregateBenchmark extends BenchmarkBase { * Actually run the benchmark, optionally specifying whether to cache the dataset. */ def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { - val ds = sparkSession.sql("select k, sum(id) from test group by k") + val ds = sparkSession.sql("select count(k), count(id) from test") val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } val prepare = () => { params.foreach { case (k, v) => sparkSession.conf.set(k, v) } From 3d48f2db8c0b0c5aec625fd00d3ed7c80c573f38 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 14:57:56 -0700 Subject: [PATCH 08/40] Codegen memory scan using ColumnarBatches --- .../columnar/InMemoryTableScanExec.scala | 108 ++++++++++++++++-- .../benchmark/AggregateBenchmark.scala | 7 +- 2 files changed, 104 insertions(+), 11 deletions(-) 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 57ca6eadf246..ef2caf22459c 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 @@ -21,19 +21,20 @@ import org.apache.spark.rdd.RDD 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.expressions.codegen.{CodegenContext, ExprCode} 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.{CodegenSupport, LeafExecNode, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized.ColumnarBatch -import org.apache.spark.sql.types.UserDefinedType +import org.apache.spark.sql.types.{DataType, UserDefinedType} case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) - extends LeafExecNode { + extends LeafExecNode with CodegenSupport { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren @@ -223,14 +224,105 @@ case class InMemoryTableScanExec( } } +// protected override def doExecute(): RDD[InternalRow] = { +// val useColumnarScan = relation.child.sqlContext.conf.getConfString( +// "spark.sql.inMemoryColumnarScan", "true").toBoolean +// if (useColumnarScan) { +// doExecuteColumnar() +// } else { +// doExecuteRow() +// } +// } + + private val useColumnarScan = relation.child.sqlContext.conf.getConfString( + "spark.sql.inMemoryColumnarScan", "true").toBoolean + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + // HACK ALERT + Seq(relation.cachedColumnVectors.asInstanceOf[RDD[InternalRow]]) + } + + override def supportCodegen: Boolean = useColumnarScan + protected override def doExecute(): RDD[InternalRow] = { - val useColumnarScan = relation.child.sqlContext.conf.getConfString( - "spark.sql.inMemoryColumnarScan", "true").toBoolean - if (useColumnarScan) { - doExecuteColumnar() + assert(!useColumnarScan, "this should only be used as a backup") + doExecuteRow() + } + + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = ctx.javaType(dataType) + val value = ctx.getValue(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { + s""" + boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); + $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); + """ } else { - doExecuteRow() + s"$javaType ${valueVar} = $value;" + }).trim + ExprCode(code, isNullVar, valueVar) + } + + // Support codegen so that we can avoid the UnsafeRow conversion in all cases. Codegen + // never requires UnsafeRow as input. + override protected def doProduce(ctx: CodegenContext): String = { + val input = ctx.freshName("input") + // PhysicalRDD always just has one input + ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") + + val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" + val batch = ctx.freshName("batch") + ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") + + val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" + val idx = ctx.freshName("batchIdx") + ctx.addMutableState("int", idx, s"$idx = 0;") + val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) + val columnAssigns = colVars.zipWithIndex.map { case (name, i) => + ctx.addMutableState(columnVectorClz, name, s"$name = null;") + s"$name = $batch.column($i);" + } + + val nextBatch = ctx.freshName("nextBatch") + ctx.addNewFunction(nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | long getBatchStart = System.nanoTime(); + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | } + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) } + s""" + |if ($batch == null) { + | $nextBatch(); + |} + |while ($batch != null) { + | int numRows = $batch.numRows(); + | while ($idx < numRows) { + | int $rowidx = $idx++; + | ${consume(ctx, columnsBatchInput).trim} + | if (shouldStop()) return; + | } + | $batch = null; + | $nextBatch(); + |} + """.stripMargin } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 142b96b5499a..2583bcbb319b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -136,11 +136,12 @@ class AggregateBenchmark extends BenchmarkBase { test("cache with randomized keys") { val N = 20 << 20 val numIters = 10 - val benchmark = new Benchmark("Cache aggregate", N) + val benchmark = new Benchmark("Cache random keys", N) sparkSession.range(N) .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - val expectedAnswer = sparkSession.sql("select count(k), count(id) from test").collect().toSeq + val query = "select count(k), count(id) from test" + val expectedAnswer = sparkSession.sql(query).collect().toSeq /** * Call collect on the dataset after deleting all existing temporary files. @@ -161,7 +162,7 @@ class AggregateBenchmark extends BenchmarkBase { * Actually run the benchmark, optionally specifying whether to cache the dataset. */ def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { - val ds = sparkSession.sql("select count(k), count(id) from test") + val ds = sparkSession.sql(query) val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } val prepare = () => { params.foreach { case (k, v) => sparkSession.conf.set(k, v) } From 9fa5b94c71bdfc61858d6ba64740fc22a9901849 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 16:10:41 -0700 Subject: [PATCH 09/40] Clean up the code a little --- .../execution/vectorized/ColumnVector.java | 10 +- .../vectorized/OnHeapColumnVector.java | 26 ---- .../execution/columnar/InMemoryRelation.scala | 6 + .../columnar/InMemoryTableScanExec.scala | 123 +++++++----------- .../benchmark/AggregateBenchmark.scala | 12 +- 5 files changed, 64 insertions(+), 113 deletions(-) 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 76e92d41b0ca..6ec6352e097c 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 @@ -16,9 +16,6 @@ */ package org.apache.spark.sql.execution.vectorized; -import java.io.Externalizable; -import java.io.ObjectInput; -import java.io.ObjectOutput; import java.math.BigDecimal; import java.math.BigInteger; @@ -60,10 +57,7 @@ * * ColumnVectors are intended to be reused. */ -public abstract class ColumnVector implements AutoCloseable {// , Externalizable { - -// public void writeExternal(ObjectOutput out) throws java.io.IOException { } -// public void readExternal(ObjectInput in) throws java.io.IOException { } +public abstract class ColumnVector implements AutoCloseable { /** * Allocates a column to store elements of `type` on or off heap. @@ -904,7 +898,7 @@ public final int appendStruct(boolean isNull) { /** * Data type for this column. */ - protected DataType type; + protected final DataType type; /** * Number of nulls in this column. This is an optimization for the reader, to skip NULL checks. diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index fe5857ff5950..9b410bacff5d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -16,8 +16,6 @@ */ package org.apache.spark.sql.execution.vectorized; -import java.io.ObjectInput; -import java.io.ObjectOutput; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; @@ -32,30 +30,6 @@ */ public final class OnHeapColumnVector extends ColumnVector { -// // capacity, num longs, long1, long2, long3... - public void writeExternal(ObjectOutput out) throws java.io.IOException { - out.writeInt(capacity); - out.writeInt(longData.length); - int i; - for (i = 0; i < longData.length; i++) { - out.writeLong(longData[i]); - } - } - - public void readExternal(ObjectInput in) throws java.io.IOException { - capacity = in.readInt(); - int numLongs = in.readInt(); - longData = new long[numLongs]; - int i; - for (i = 0; i < numLongs; i++) { - longData[i] = in.readLong(); - } - } - - public OnHeapColumnVector() { - super(10000, LongType$.MODULE$, MemoryMode.ON_HEAP); - } - private static final boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); 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 a83217dcb256..dc3f9f920851 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 @@ -96,6 +96,9 @@ case class InMemoryRelation( buildColumnarBuffers() } + /** + * Batch the input rows using [[ColumnBuilder]]s. + */ private def buildBuffers(): Unit = { val output = child.output val cached = child.execute().mapPartitionsInternal { rowIterator => @@ -150,6 +153,9 @@ case class InMemoryRelation( _cachedColumnBuffers = cached } + /** + * Batch the input rows using [[ColumnarBatch]]es. + */ private def buildColumnarBuffers(): Unit = { val schema = StructType.fromAttributes(child.output) val cached = child.execute().mapPartitionsInternal { rowIterator => 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 ef2caf22459c..3dff9db3c810 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 @@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized.ColumnarBatch +import org.apache.spark.sql.execution.vectorized.{ColumnVector, ColumnarBatch} import org.apache.spark.sql.types.{DataType, UserDefinedType} @@ -36,6 +36,17 @@ case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode with CodegenSupport { + private val useColumnarScan = relation.child.sqlContext.conf.getConfString( + "spark.sql.inMemoryColumnarScan", "true").toBoolean + + override val supportCodegen: Boolean = useColumnarScan + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + // HACK ALERT: This is actually an RDD[ColumnarBatch]. + // We're taking advantage of Scala's type erasure here to pass these batches along. + Seq(relation.cachedColumnVectors.asInstanceOf[RDD[InternalRow]]) + } + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren override lazy val metrics = Map( @@ -119,7 +130,8 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning - private def doExecuteRow(): RDD[InternalRow] = { + override protected def doExecute(): RDD[InternalRow] = { + assert(!useColumnarScan) val numOutputRows = longMetric("numOutputRows") if (enableAccumulators) { @@ -198,81 +210,10 @@ case class InMemoryTableScanExec( } } - private def doExecuteColumnar(): RDD[InternalRow] = { - relation.cachedColumnVectors.mapPartitionsInternal { batchIter => - new Iterator[InternalRow] { - private var currentRowIterator: java.util.Iterator[ColumnarBatch.Row] = null - - override def hasNext: Boolean = { - if (currentRowIterator == null && batchIter.hasNext) { - currentRowIterator = batchIter.next().rowIterator - } - currentRowIterator != null && currentRowIterator.hasNext - } - - override def next(): InternalRow = { - if (!hasNext) { - throw new NoSuchElementException - } - val row = currentRowIterator.next() - if (!currentRowIterator.hasNext) { - currentRowIterator = null - } - row - } - } - } - } - -// protected override def doExecute(): RDD[InternalRow] = { -// val useColumnarScan = relation.child.sqlContext.conf.getConfString( -// "spark.sql.inMemoryColumnarScan", "true").toBoolean -// if (useColumnarScan) { -// doExecuteColumnar() -// } else { -// doExecuteRow() -// } -// } - - private val useColumnarScan = relation.child.sqlContext.conf.getConfString( - "spark.sql.inMemoryColumnarScan", "true").toBoolean - - override def inputRDDs(): Seq[RDD[InternalRow]] = { - // HACK ALERT - Seq(relation.cachedColumnVectors.asInstanceOf[RDD[InternalRow]]) - } - - override def supportCodegen: Boolean = useColumnarScan - - protected override def doExecute(): RDD[InternalRow] = { - assert(!useColumnarScan, "this should only be used as a backup") - doExecuteRow() - } - - private def genCodeColumnVector( - ctx: CodegenContext, - columnVar: String, - ordinal: String, - dataType: DataType, - nullable: Boolean): ExprCode = { - val javaType = ctx.javaType(dataType) - val value = ctx.getValue(columnVar, dataType, ordinal) - val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } - val valueVar = ctx.freshName("value") - val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { - s""" - boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); - $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); - """ - } else { - s"$javaType ${valueVar} = $value;" - }).trim - ExprCode(code, isNullVar, valueVar) - } - - // Support codegen so that we can avoid the UnsafeRow conversion in all cases. Codegen - // never requires UnsafeRow as input. + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. + * This produces an [[UnsafeRow]] for each row in each batch. + */ override protected def doProduce(ctx: CodegenContext): String = { val input = ctx.freshName("input") // PhysicalRDD always just has one input @@ -325,4 +266,30 @@ case class InMemoryTableScanExec( """.stripMargin } + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. + * This is called once per [[ColumnarBatch]]. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = ctx.javaType(dataType) + val value = ctx.getValue(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { + s""" + boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); + $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); + """ + } else { + s"$javaType ${valueVar} = $value;" + }).trim + ExprCode(code, isNullVar, valueVar) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 2583bcbb319b..c63bdb5a5ace 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -159,7 +159,7 @@ class AggregateBenchmark extends BenchmarkBase { } /** - * Actually run the benchmark, optionally specifying whether to cache the dataset. + * Add a benchmark case, optionally specifying whether to cache the dataset. */ def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { val ds = sparkSession.sql(query) @@ -189,6 +189,16 @@ class AggregateBenchmark extends BenchmarkBase { f() } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + -------------------------------------------------------------------------------------------- + cache = F 906 / 1160 23.2 43.2 1.0X + cache = T columnar = F compress = F 1141 / 1234 18.4 54.4 0.8X + cache = T columnar = T compress = F 535 / 797 39.2 25.5 1.7X + */ + benchmark.run() } From 631c3b4cd7f077ac4b15f293d03d456b8b4e69e5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 16:19:42 -0700 Subject: [PATCH 10/40] Clean up a little more --- .../spark/sql/execution/vectorized/ColumnVector.java | 1 - .../spark/sql/execution/columnar/InMemoryRelation.scala | 6 ++++++ .../sql/execution/columnar/InMemoryTableScanExec.scala | 8 ++++++-- 3 files changed, 12 insertions(+), 3 deletions(-) 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 6ec6352e097c..354c878aca00 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 @@ -58,7 +58,6 @@ * ColumnVectors are intended to be reused. */ public abstract class ColumnVector implements AutoCloseable { - /** * Allocates a column to store elements of `type` on or off heap. * Capacity is the initial capacity of the vector and it will grow as necessary. Capacity is 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 dc3f9f920851..eeaf9f1bfc41 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 @@ -86,12 +86,17 @@ case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { buildBuffers() + } + + if (_cachedColumnVectors == null) { buildColumnarBuffers() } def recache(): Unit = { _cachedColumnBuffers.unpersist() + _cachedColumnVectors.unpersist() _cachedColumnBuffers = null + _cachedColumnVectors = null buildBuffers() buildColumnarBuffers() } @@ -173,6 +178,7 @@ case class InMemoryRelation( var i = 0 totalSize = 0 while (i < row.numFields) { + // HACK ALERT: This doesn't work for other data types! :) columnarBatch.column(i).putLong(rowCount, row.getLong(i)) totalSize += 8 i += 1 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 3dff9db3c810..2cab20aad7e3 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode} +import org.apache.spark.sql.execution.{BatchedDataSourceScanExec, CodegenSupport, LeafExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized.{ColumnVector, ColumnarBatch} import org.apache.spark.sql.types.{DataType, UserDefinedType} @@ -130,7 +130,7 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning - override protected def doExecute(): RDD[InternalRow] = { + protected override def doExecute(): RDD[InternalRow] = { assert(!useColumnarScan) val numOutputRows = longMetric("numOutputRows") @@ -212,7 +212,9 @@ case class InMemoryTableScanExec( /** * Produce code to process the input iterator as [[ColumnarBatch]]es. + * * This produces an [[UnsafeRow]] for each row in each batch. + * DUPLICATE CODE ALERT: This is copied directly from [[BatchedDataSourceScanExec]]. */ override protected def doProduce(ctx: CodegenContext): String = { val input = ctx.freshName("input") @@ -268,7 +270,9 @@ case class InMemoryTableScanExec( /** * Generate [[ColumnVector]] expressions for our parent to consume as rows. + * * This is called once per [[ColumnarBatch]]. + * DUPLICATE CODE ALERT: This is copied directly from [[BatchedDataSourceScanExec]]. */ private def genCodeColumnVector( ctx: CodegenContext, From fc556a69da1e2deb026cc95405b0a9764e990211 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 15:10:41 -0700 Subject: [PATCH 11/40] Generate code for write path to support other types Previously we could only support schemas where all columns are Longs because we hardcode putLong and getLong calls in the write path. This led to unfathomable NPEs if we try to cache something with other types. This commit fixes this by generalizing the code to build column batches. --- .../spark/sql/execution/CacheManager.scala | 4 +- .../columnar/GenerateColumnarBatch.scala | 133 ++++++++++++++++++ .../execution/columnar/InMemoryRelation.scala | 68 ++++----- .../columnar/InMemoryTableScanExec.scala | 12 +- 4 files changed, 170 insertions(+), 47 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 4ca134700857..4ddcbd3d396e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -114,7 +114,7 @@ class CacheManager extends Logging { val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) val found = dataIndex >= 0 if (found) { - cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData(dataIndex).cachedRepresentation.unpersist(blocking) cachedData.remove(dataIndex) } found @@ -172,7 +172,7 @@ class CacheManager extends Logging { case data if data.plan.find(lookupAndRefresh(_, fs, qualifiedPath)).isDefined => val dataIndex = cachedData.indexWhere(cd => data.plan.sameResult(cd.plan)) if (dataIndex >= 0) { - data.cachedRepresentation.cachedColumnBuffers.unpersist(blocking = true) + data.cachedRepresentation.unpersist() cachedData.remove(dataIndex) } sparkSession.sharedState.cacheManager.cacheQuery(Dataset.ofRows(sparkSession, data.plan)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala new file mode 100644 index 000000000000..296b1ead1306 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter} +import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator +import org.apache.spark.sql.execution.vectorized.ColumnarBatch +import org.apache.spark.sql.types._ + + +/** + * A helper class to expose the scala iterator to Java. + */ +abstract class ColumnarBatchIterator extends Iterator[ColumnarBatch] + + +/** + * Generate code to batch [[InternalRow]]s into [[ColumnarBatch]]es. + */ +class GenerateColumnarBatch( + schema: StructType, + batchSize: Int) + extends CodeGenerator[Iterator[InternalRow], Iterator[ColumnarBatch]] { + + protected def canonicalize(in: Iterator[InternalRow]): Iterator[InternalRow] = in + + protected def bind( + in: Iterator[InternalRow], + inputSchema: Seq[Attribute]): Iterator[InternalRow] = { + in + } + + protected def create(rowIterator: Iterator[InternalRow]): Iterator[ColumnarBatch] = { + import scala.collection.JavaConverters._ + val ctx = newCodeGenContext() + val batchVar = ctx.freshName("columnarBatch") + val rowNumVar = ctx.freshName("rowNum") + val numBytesVar = ctx.freshName("bytesInBatch") + val rowIterVar = ctx.addReferenceObj( + "rowIterator", rowIterator.asJava, classOf[java.util.Iterator[_]].getName) + val schemaVar = ctx.addReferenceObj("schema", schema, classOf[StructType].getName) + val maxNumBytes = ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE + val populateColumnVectorsCode = schema.fields.zipWithIndex.map { case (field, i) => + val typeName = GenerateColumnarBatch.typeToName(field.dataType) + val put = "put" + typeName.capitalize + val get = "get" + typeName.capitalize + s""" + $batchVar.column($i).$put($rowNumVar, row.$get($i)); + $numBytesVar += ${field.dataType.defaultSize}; + """.trim + }.mkString("\n") + val code = s""" + import org.apache.spark.memory.MemoryMode; + import org.apache.spark.sql.catalyst.InternalRow; + import org.apache.spark.sql.execution.vectorized.ColumnarBatch; + + public GeneratedColumnarBatchIterator generate(Object[] references) { + return new GeneratedColumnarBatchIterator(references); + } + + class GeneratedColumnarBatchIterator extends ${classOf[ColumnarBatchIterator].getName} { + ${ctx.declareMutableStates()} + + public GeneratedColumnarBatchIterator(Object[] references) { + ${ctx.initMutableStates()} + } + + @Override + public boolean hasNext() { + return $rowIterVar.hasNext(); + } + + @Override + public ColumnarBatch next() { + ColumnarBatch $batchVar = + ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP, $batchSize); + int $rowNumVar = 0; + long $numBytesVar = 0; + while ($rowIterVar.hasNext() && $rowNumVar < $batchSize && $numBytesVar < $maxNumBytes) { + InternalRow row = (InternalRow) $rowIterVar.next(); + $populateColumnVectorsCode + $rowNumVar += 1; + } + $batchVar.setNumRows($rowNumVar); + return $batchVar; + } + } + """ + val formattedCode = CodeFormatter.stripOverlappingComments( + new CodeAndComment(code, ctx.getPlaceHolderToComments())) + CodeGenerator.compile(formattedCode).generate(ctx.references.toArray) + .asInstanceOf[Iterator[ColumnarBatch]] + } + +} + + +private[columnar] object GenerateColumnarBatch { + + private val typeToName = Map[DataType, String]( + BooleanType -> "boolean", + ByteType -> "byte", + ShortType -> "short", + IntegerType -> "int", + LongType -> "long", + FloatType -> "float", + DoubleType -> "double") + + /** + * Whether [[ColumnarBatch]]-based caching is supported for the given data type + */ + def isSupported(dataType: DataType): Boolean = { + typeToName.contains(dataType) + } + +} 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 eeaf9f1bfc41..4d3148c29521 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils -import org.apache.spark.memory.MemoryMode import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystConf, InternalRow} @@ -66,6 +65,14 @@ case class InMemoryRelation( val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { + // Fallback to using ColumnBuilders if the schema has non-primitive types + private[columnar] val useColumnBatches = { + val enabled = child.sqlContext.conf.getConfString( + "spark.sql.inMemoryColumnarScan", "true").toBoolean + val supported = output.forall { a => GenerateColumnarBatch.isSupported(a.dataType) } + enabled && supported + } + override protected def innerChildren: Seq[SparkPlan] = Seq(child) override def producedAttributes: AttributeSet = outputSet @@ -84,21 +91,32 @@ case class InMemoryRelation( // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. - if (_cachedColumnBuffers == null) { + if (useColumnBatches && _cachedColumnVectors == null) { + buildColumnarBuffers() + } + + if (!useColumnBatches && _cachedColumnBuffers == null) { buildBuffers() } - if (_cachedColumnVectors == null) { - buildColumnarBuffers() + def unpersist(blocking: Boolean = true): Unit = { + if (_cachedColumnBuffers != null) { + _cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null + } + if (_cachedColumnVectors != null) { + _cachedColumnVectors.unpersist(blocking) + _cachedColumnVectors = null + } } def recache(): Unit = { - _cachedColumnBuffers.unpersist() - _cachedColumnVectors.unpersist() - _cachedColumnBuffers = null - _cachedColumnVectors = null - buildBuffers() - buildColumnarBuffers() + unpersist() + if (useColumnBatches) { + buildColumnarBuffers() + } else { + buildBuffers() + } } /** @@ -160,39 +178,13 @@ case class InMemoryRelation( /** * Batch the input rows using [[ColumnarBatch]]es. + * This provides a faster implementation of in-memory scan. */ private def buildColumnarBuffers(): Unit = { val schema = StructType.fromAttributes(child.output) val cached = child.execute().mapPartitionsInternal { rowIterator => - new Iterator[ColumnarBatch] { - def next(): ColumnarBatch = { - val columnarBatch = ColumnarBatch.allocate(schema, MemoryMode.ON_HEAP, batchSize) - var rowCount = 0 - var totalSize = 0L - while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { - val row = rowIterator.next() - assert(row.numFields == columnarBatch.numCols, "Row column number mismatch, " + - s"expected ${columnarBatch.numCols} columns, but got ${row.numFields}. \n" + - s"Row content: $row") - var i = 0 - totalSize = 0 - while (i < row.numFields) { - // HACK ALERT: This doesn't work for other data types! :) - columnarBatch.column(i).putLong(rowCount, row.getLong(i)) - totalSize += 8 - i += 1 - } - rowCount += 1 - } - columnarBatch.setNumRows(rowCount) - columnarBatch - } - - def hasNext: Boolean = rowIterator.hasNext - } + new GenerateColumnarBatch(schema, batchSize).generate(rowIterator) }.persist(storageLevel) - cached.setName( tableName.map(n => s"In-memory table $n") .getOrElse(StringUtils.abbreviate(child.toString, 1024))) 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 2cab20aad7e3..52dcf56cb3bf 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 @@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{BatchedDataSourceScanExec, CodegenSupport, LeafExecNode} +import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.sql.types.{DataType, UserDefinedType} @@ -36,10 +36,7 @@ case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode with CodegenSupport { - private val useColumnarScan = relation.child.sqlContext.conf.getConfString( - "spark.sql.inMemoryColumnarScan", "true").toBoolean - - override val supportCodegen: Boolean = useColumnarScan + override val supportCodegen: Boolean = relation.useColumnBatches override def inputRDDs(): Seq[RDD[InternalRow]] = { // HACK ALERT: This is actually an RDD[ColumnarBatch]. @@ -131,7 +128,8 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning protected override def doExecute(): RDD[InternalRow] = { - assert(!useColumnarScan) + assert(!relation.useColumnBatches) + assert(relation.cachedColumnBuffers != null) val numOutputRows = longMetric("numOutputRows") if (enableAccumulators) { From 9091f93e97bd16090b863b0dfbde24fe6d4f6e89 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 16:01:44 -0700 Subject: [PATCH 12/40] Move cache benchmark to new file --- .../spark/scheduler/TaskSetManager.scala | 2 +- .../benchmark/AggregateBenchmark.scala | 64 +++------- .../execution/benchmark/BenchmarkBase.scala | 4 +- .../execution/benchmark/CacheBenchmark.scala | 113 ++++++++++++++++++ 4 files changed, 131 insertions(+), 52 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index af6d86908cd3..c7ff13cebfaa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -738,7 +738,7 @@ private[spark] class TaskSetManager( s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { case fetchFailed: FetchFailed => - // logWarning(failureReason) + logWarning(failureReason) if (!successful(index)) { successful(index) = true tasksSuccessful += 1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index c63bdb5a5ace..5b8fda47b9ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -21,7 +21,6 @@ import java.util.HashMap import org.apache.spark.SparkConf import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} -import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap @@ -132,48 +131,18 @@ class AggregateBenchmark extends BenchmarkBase { */ } + ignore("aggregate with randomized keys") { + val N = 20 << 22 - test("cache with randomized keys") { - val N = 20 << 20 - val numIters = 10 - val benchmark = new Benchmark("Cache random keys", N) - sparkSession.range(N) - .selectExpr("id", "floor(rand() * 10000) as k") + val benchmark = new Benchmark("Aggregate w keys", N) + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - val query = "select count(k), count(id) from test" - val expectedAnswer = sparkSession.sql(query).collect().toSeq - /** - * Call collect on the dataset after deleting all existing temporary files. - */ - def doCollect(df: org.apache.spark.sql.DataFrame): Unit = { - df.sparkSession.sparkContext.parallelize(1 to 10, 10).foreach { _ => - org.apache.spark.SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach { dir => - dir.delete() - } - } - QueryTest.checkAnswer(df, expectedAnswer) match { - case Some(errMessage) => throw new RuntimeException(errMessage) - case None => // all good - } - } + def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() - /** - * Add a benchmark case, optionally specifying whether to cache the dataset. - */ - def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { - val ds = sparkSession.sql(query) - val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } - val prepare = () => { - params.foreach { case (k, v) => sparkSession.conf.set(k, v) } - if (cache) { sparkSession.catalog.cacheTable("test") } - doCollect(ds) - } - val cleanup = () => { - defaults.foreach { case (k, v) => sparkSession.conf.set(k, v) } - sparkSession.catalog.clearCache() - } - benchmark.addCase(name, numIters, prepare, cleanup) { _ => doCollect(ds) } + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => @@ -190,16 +159,15 @@ class AggregateBenchmark extends BenchmarkBase { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - cache = F 906 / 1160 23.2 43.2 1.0X - cache = T columnar = F compress = F 1141 / 1234 18.4 54.4 0.8X - cache = T columnar = T compress = F 535 / 797 39.2 25.5 1.7X - */ + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - benchmark.run() + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 7445 / 7517 11.3 88.7 1.0X + codegen = T hashmap = F 4672 / 4703 18.0 55.7 1.6X + codegen = T hashmap = T 1764 / 1958 47.6 21.0 4.2X + */ } ignore("aggregate with string key") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala index 96445beec6fb..c99a5aec1cd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala @@ -28,10 +28,8 @@ import org.apache.spark.util.Benchmark private[benchmark] trait BenchmarkBase extends SparkFunSuite { lazy val sparkSession = SparkSession.builder - .master("local-cluster[2,1,1024]") + .master("local[1]") .appName("microbenchmark") - .config("spark.ui.enabled", true) - .config("spark.ui.port", 5050) .config("spark.sql.shuffle.partitions", 1) .config("spark.sql.autoBroadcastJoinThreshold", 1) .getOrCreate() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala new file mode 100644 index 000000000000..c8923d957ad0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.SparkEnv +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.util.Benchmark + + +class CacheBenchmark extends BenchmarkBase { + + /** + * Call collect on a [[DataFrame]] after deleting all existing temporary files. + * This also checks whether the collected result matches the expected answer. + */ + private def collect(df: DataFrame, expectedAnswer: Seq[Row]): Unit = { + df.sparkSession.sparkContext.parallelize(1 to 10, 10).foreach { _ => + SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach { dir => + dir.delete() + } + } + QueryTest.checkAnswer(df, expectedAnswer) match { + case Some(errMessage) => throw new RuntimeException(errMessage) + case None => // all good + } + } + + /* + * NOTE: When running this benchmark, you will get a lot of WARN logs complaining that the + * shuffle files do not exist. This is intentional; we delete the shuffle files manually + * after every call to `collect` to avoid the next run to reuse shuffle files written by + * the previous run. + */ + test("cache with randomized keys") { + val N = 20 << 21 + val numIters = 10 + val benchmark = new Benchmark("Cache random keys", N) + sparkSession.range(N) + .selectExpr("id", "floor(rand() * 10000) as k") + .createOrReplaceTempView("test") + val query = "select count(k), count(id) from test" + val expectedAnswer = sparkSession.sql(query).collect().toSeq + + /** + * Add a benchmark case, optionally specifying whether to cache the dataset. + */ + def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { + val ds = sparkSession.sql(query) + val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } + val prepare = () => { + params.foreach { case (k, v) => sparkSession.conf.set(k, v) } + if (cache) { sparkSession.catalog.cacheTable("test") } + collect(ds, expectedAnswer) + } + val cleanup = () => { + defaults.foreach { case (k, v) => sparkSession.conf.set(k, v) } + sparkSession.catalog.clearCache() + } + benchmark.addCase(name, numIters, prepare, cleanup) { _ => collect(ds, expectedAnswer) } + } + + // All of these are codegen = T hashmap = T + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "1024") + + // Benchmark cases: + // (1) No caching + // (2) Caching without compression + // (3) Caching with compression + // (4) Caching with column batches (without compression) + addBenchmark("cache = F", cache = false) + addBenchmark("cache = T columnar_batches = F compress = F", cache = true, Map( + "spark.sql.inMemoryColumnarScan" -> "false", + "spark.sql.inMemoryColumnarStorage.compressed" -> "false" + )) + addBenchmark("cache = T columnar_batches = F compress = T", cache = true, Map( + "spark.sql.inMemoryColumnarScan" -> "false", + "spark.sql.inMemoryColumnarStorage.compressed" -> "true" + )) + addBenchmark("cache = T columnar_batches = T", cache = true, Map( + "spark.sql.inMemoryColumnarScan" -> "true" + )) + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ----------------------------------------------------------------------------------------------- + cache = F 890 / 920 47.1 21.2 1.0X + cache = T columnar_batches = F compress = F 1950 / 1978 21.5 46.5 0.5X + cache = T columnar_batches = F compress = T 1893 / 1927 22.2 45.1 0.5X + cache = T columnar_batches = T 540 / 544 77.7 12.9 1.6X + */ + } + +} From 6bc3defe1097a984712976d31e6394d1e5e83663 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 16:43:13 -0700 Subject: [PATCH 13/40] Abstract codegen code into ColumnarBatchScan --- .../columnar/InMemoryTableScanExec.scala | 94 +------------------ 1 file changed, 3 insertions(+), 91 deletions(-) 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 52dcf56cb3bf..66532d8192bb 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 @@ -21,20 +21,18 @@ import org.apache.spark.rdd.RDD 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.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode} +import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} -import org.apache.spark.sql.types.{DataType, UserDefinedType} +import org.apache.spark.sql.types.UserDefinedType case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) - extends LeafExecNode with CodegenSupport { + extends LeafExecNode with ColumnarBatchScan { override val supportCodegen: Boolean = relation.useColumnBatches @@ -208,90 +206,4 @@ case class InMemoryTableScanExec( } } - /** - * Produce code to process the input iterator as [[ColumnarBatch]]es. - * - * This produces an [[UnsafeRow]] for each row in each batch. - * DUPLICATE CODE ALERT: This is copied directly from [[BatchedDataSourceScanExec]]. - */ - override protected def doProduce(ctx: CodegenContext): String = { - val input = ctx.freshName("input") - // PhysicalRDD always just has one input - ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") - - val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" - val batch = ctx.freshName("batch") - ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") - - val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" - val idx = ctx.freshName("batchIdx") - ctx.addMutableState("int", idx, s"$idx = 0;") - val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) - val columnAssigns = colVars.zipWithIndex.map { case (name, i) => - ctx.addMutableState(columnVectorClz, name, s"$name = null;") - s"$name = $batch.column($i);" - } - - val nextBatch = ctx.freshName("nextBatch") - ctx.addNewFunction(nextBatch, - s""" - |private void $nextBatch() throws java.io.IOException { - | long getBatchStart = System.nanoTime(); - | if ($input.hasNext()) { - | $batch = ($columnarBatchClz)$input.next(); - | $idx = 0; - | ${columnAssigns.mkString("", "\n", "\n")} - | } - |}""".stripMargin) - - ctx.currentVars = null - val rowidx = ctx.freshName("rowIdx") - val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => - genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) - } - s""" - |if ($batch == null) { - | $nextBatch(); - |} - |while ($batch != null) { - | int numRows = $batch.numRows(); - | while ($idx < numRows) { - | int $rowidx = $idx++; - | ${consume(ctx, columnsBatchInput).trim} - | if (shouldStop()) return; - | } - | $batch = null; - | $nextBatch(); - |} - """.stripMargin - } - - /** - * Generate [[ColumnVector]] expressions for our parent to consume as rows. - * - * This is called once per [[ColumnarBatch]]. - * DUPLICATE CODE ALERT: This is copied directly from [[BatchedDataSourceScanExec]]. - */ - private def genCodeColumnVector( - ctx: CodegenContext, - columnVar: String, - ordinal: String, - dataType: DataType, - nullable: Boolean): ExprCode = { - val javaType = ctx.javaType(dataType) - val value = ctx.getValue(columnVar, dataType, ordinal) - val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } - val valueVar = ctx.freshName("value") - val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { - s""" - boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); - $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); - """ - } else { - s"$javaType ${valueVar} = $value;" - }).trim - ExprCode(code, isNullVar, valueVar) - } - } From 0e79dbefbad655289f9359e52852ba25314b39f0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:04:09 -0700 Subject: [PATCH 14/40] Introduce CACHE_CODEGEN config to reduce dup code --- .../columnar/GenerateColumnarBatch.scala | 1 + .../execution/columnar/InMemoryRelation.scala | 24 +++++++++---------- .../columnar/InMemoryTableScanExec.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 7 ++++++ .../execution/benchmark/CacheBenchmark.scala | 15 ++++++------ 5 files changed, 29 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 296b1ead1306..1becca162fca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -57,6 +57,7 @@ class GenerateColumnarBatch( "rowIterator", rowIterator.asJava, classOf[java.util.Iterator[_]].getName) val schemaVar = ctx.addReferenceObj("schema", schema, classOf[StructType].getName) val maxNumBytes = ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE + // Code to populate column vectors with the values of the input rows val populateColumnVectorsCode = schema.fields.zipWithIndex.map { case (field, i) => val typeName = GenerateColumnarBatch.typeToName(field.dataType) val put = "put" + typeName.capitalize 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 4d3148c29521..7c03b86eda0e 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 @@ -28,6 +28,7 @@ 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.ColumnarBatch +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel import org.apache.spark.util.LongAccumulator @@ -66,9 +67,8 @@ case class InMemoryRelation( extends logical.LeafNode with MultiInstanceRelation { // Fallback to using ColumnBuilders if the schema has non-primitive types - private[columnar] val useColumnBatches = { - val enabled = child.sqlContext.conf.getConfString( - "spark.sql.inMemoryColumnarScan", "true").toBoolean + private[columnar] val useColumnBatches: Boolean = { + val enabled = child.sqlContext.conf.getConf(SQLConf.CACHE_CODEGEN) val supported = output.forall { a => GenerateColumnarBatch.isSupported(a.dataType) } enabled && supported } @@ -91,8 +91,8 @@ case class InMemoryRelation( // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. - if (useColumnBatches && _cachedColumnVectors == null) { - buildColumnarBuffers() + if (useColumnBatches && _cachedColumnBatches == null) { + buildColumnarBatches() } if (!useColumnBatches && _cachedColumnBuffers == null) { @@ -104,16 +104,16 @@ case class InMemoryRelation( _cachedColumnBuffers.unpersist(blocking) _cachedColumnBuffers = null } - if (_cachedColumnVectors != null) { - _cachedColumnVectors.unpersist(blocking) - _cachedColumnVectors = null + if (_cachedColumnBatches != null) { + _cachedColumnBatches.unpersist(blocking) + _cachedColumnBatches = null } } def recache(): Unit = { unpersist() if (useColumnBatches) { - buildColumnarBuffers() + buildColumnarBatches() } else { buildBuffers() } @@ -180,7 +180,7 @@ case class InMemoryRelation( * Batch the input rows using [[ColumnarBatch]]es. * This provides a faster implementation of in-memory scan. */ - private def buildColumnarBuffers(): Unit = { + private def buildColumnarBatches(): Unit = { val schema = StructType.fromAttributes(child.output) val cached = child.execute().mapPartitionsInternal { rowIterator => new GenerateColumnarBatch(schema, batchSize).generate(rowIterator) @@ -188,7 +188,7 @@ case class InMemoryRelation( cached.setName( tableName.map(n => s"In-memory table $n") .getOrElse(StringUtils.abbreviate(child.toString, 1024))) - _cachedColumnVectors = cached + _cachedColumnBatches = cached } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { @@ -210,7 +210,7 @@ case class InMemoryRelation( } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers - def cachedColumnVectors: RDD[ColumnarBatch] = _cachedColumnVectors + def cachedColumnBatches: RDD[ColumnarBatch] = _cachedColumnBatches override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, batchStats) 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 66532d8192bb..71889920f98f 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 @@ -39,7 +39,7 @@ case class InMemoryTableScanExec( override def inputRDDs(): Seq[RDD[InternalRow]] = { // HACK ALERT: This is actually an RDD[ColumnarBatch]. // We're taking advantage of Scala's type erasure here to pass these batches along. - Seq(relation.cachedColumnVectors.asInstanceOf[RDD[InternalRow]]) + Seq(relation.cachedColumnBatches.asInstanceOf[RDD[InternalRow]]) } override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren 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 645b0fa13ee3..21ab6fdd9ac5 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 @@ -90,6 +90,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val CACHE_CODEGEN = SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.codegen") + .internal() + .doc("When true, use generated code to build column batches for caching. This is only " + + "supported for basic types and improves caching performance for such types.") + .booleanConf + .createWithDefault(true) + val PREFER_SORTMERGEJOIN = SQLConfigBuilder("spark.sql.join.preferSortMergeJoin") .internal() .doc("When true, prefer sort merge join over shuffle hash join.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala index c8923d957ad0..b337c749d08d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.SparkEnv import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Benchmark @@ -75,8 +76,8 @@ class CacheBenchmark extends BenchmarkBase { } // All of these are codegen = T hashmap = T - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "1024") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.VECTORIZED_AGG_MAP_MAX_COLUMNS.key, "1024") // Benchmark cases: // (1) No caching @@ -85,15 +86,15 @@ class CacheBenchmark extends BenchmarkBase { // (4) Caching with column batches (without compression) addBenchmark("cache = F", cache = false) addBenchmark("cache = T columnar_batches = F compress = F", cache = true, Map( - "spark.sql.inMemoryColumnarScan" -> "false", - "spark.sql.inMemoryColumnarStorage.compressed" -> "false" + SQLConf.CACHE_CODEGEN.key -> "false", + SQLConf.COMPRESS_CACHED.key -> "false" )) addBenchmark("cache = T columnar_batches = F compress = T", cache = true, Map( - "spark.sql.inMemoryColumnarScan" -> "false", - "spark.sql.inMemoryColumnarStorage.compressed" -> "true" + SQLConf.CACHE_CODEGEN.key -> "false", + SQLConf.COMPRESS_CACHED.key -> "true" )) addBenchmark("cache = T columnar_batches = T", cache = true, Map( - "spark.sql.inMemoryColumnarScan" -> "true" + SQLConf.CACHE_CODEGEN.key -> "true" )) benchmark.run() From 08bb8a591ac483183565bd80c92aa648912ee572 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:34:57 -0700 Subject: [PATCH 15/40] Add some tests for InMemoryRelation --- .../columnar/InMemoryTableScanExec.scala | 11 +++-- .../columnar/InMemoryColumnarQuerySuite.scala | 42 +++++++++++++++++++ 2 files changed, 49 insertions(+), 4 deletions(-) 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 71889920f98f..ec0a5fedcef1 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 @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode} -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType @@ -37,9 +36,13 @@ case class InMemoryTableScanExec( override val supportCodegen: Boolean = relation.useColumnBatches override def inputRDDs(): Seq[RDD[InternalRow]] = { - // HACK ALERT: This is actually an RDD[ColumnarBatch]. - // We're taking advantage of Scala's type erasure here to pass these batches along. - Seq(relation.cachedColumnBatches.asInstanceOf[RDD[InternalRow]]) + if (relation.useColumnBatches) { + // HACK ALERT: This is actually an RDD[ColumnarBatch]. + // We're taking advantage of Scala's type erasure here to pass these batches along. + Seq(relation.cachedColumnBatches.asInstanceOf[RDD[InternalRow]]) + } else { + Seq() + } } override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index f355a5200ce2..ee972e3d37b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ @@ -390,4 +391,45 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } } + test("InMemoryRelation builds the correct buffers") { + testColumnBatches(useColumnBatches = true, useComplexSchema = false) + testColumnBatches(useColumnBatches = false, useComplexSchema = false) + } + + test("InMemoryRelation falls back on non-codegen path with complex schemas") { + testColumnBatches(useColumnBatches = true, useComplexSchema = true) + testColumnBatches(useColumnBatches = false, useComplexSchema = true) + } + + private def testColumnBatches(useColumnBatches: Boolean, useComplexSchema: Boolean = false) { + withSQLConf(SQLConf.CACHE_CODEGEN.key -> useColumnBatches.toString) { + val logicalPlan = org.apache.spark.sql.catalyst.plans.logical.Range(1, 10, 1, 10) + val sparkPlan = new org.apache.spark.sql.execution.RangeExec(logicalPlan) { + override val output: Seq[Attribute] = { + if (useComplexSchema) { + Seq(AttributeReference("complex", ArrayType(LongType))()) + } else { + logicalPlan.output + } + } + } + val inMemoryRelation = InMemoryRelation( + useCompression = false, + batchSize = 100, + storageLevel = MEMORY_ONLY, + child = sparkPlan, + tableName = None) + if (!useComplexSchema) { + assert(inMemoryRelation.useColumnBatches == useColumnBatches) + assert((inMemoryRelation.cachedColumnBatches != null) == useColumnBatches) + assert((inMemoryRelation.cachedColumnBuffers == null) == useColumnBatches) + } else { + // Fallback on using non-code-gen'ed column builders if schema is complex + assert(!inMemoryRelation.useColumnBatches) + assert(inMemoryRelation.cachedColumnBatches == null) + assert(inMemoryRelation.cachedColumnBuffers != null) + } + } + } + } From 6f2c274de45a2396ac12676b90037ad57fee67b4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:40:11 -0700 Subject: [PATCH 16/40] Add some tests for InMemoryRelation --- .../columnar/InMemoryColumnarQuerySuite.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index ee972e3d37b1..3bb829069007 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.internal.SQLConf @@ -429,6 +430,20 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(inMemoryRelation.cachedColumnBatches == null) assert(inMemoryRelation.cachedColumnBuffers != null) } + // Test unpersist and recaching + def rdd: RDD[_] = + if (useColumnBatches && !useComplexSchema) { + inMemoryRelation.cachedColumnBatches + } else { + inMemoryRelation.cachedColumnBuffers + } + assert(rdd != null) + assert(rdd.getStorageLevel == MEMORY_ONLY) + inMemoryRelation.recache() + assert(rdd.getStorageLevel == MEMORY_ONLY) + inMemoryRelation.unpersist(blocking = true) + assert(inMemoryRelation.cachedColumnBatches == null) + assert(inMemoryRelation.cachedColumnBuffers == null) } } From ac878d486c9b3f655d215609b518c246c741642f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:44:21 -0700 Subject: [PATCH 17/40] Fix InMemoryColumnarQuerySuite --- .../apache/spark/sql/execution/columnar/InMemoryRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7c03b86eda0e..a281d5294a6f 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 @@ -213,5 +213,5 @@ case class InMemoryRelation( def cachedColumnBatches: RDD[ColumnarBatch] = _cachedColumnBatches override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, batchStats) + Seq(_cachedColumnBuffers, _cachedColumnBatches, batchStats) } From 7519125cc945df3e9b5f08109f53af83e7609232 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 24 Jun 2016 12:00:37 -0700 Subject: [PATCH 18/40] Clean up code: abstract CachedBatch and ColumnarBatch --- .../spark/sql/execution/CacheManager.scala | 4 +- .../columnar/GenerateColumnAccessor.scala | 3 +- .../execution/columnar/InMemoryRelation.scala | 115 +++++++++--------- .../columnar/InMemoryTableScanExec.scala | 15 ++- .../columnar/InMemoryColumnarQuerySuite.scala | 29 +---- 5 files changed, 78 insertions(+), 88 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 4ddcbd3d396e..4ca134700857 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -114,7 +114,7 @@ class CacheManager extends Logging { val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) val found = dataIndex >= 0 if (found) { - cachedData(dataIndex).cachedRepresentation.unpersist(blocking) + cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) cachedData.remove(dataIndex) } found @@ -172,7 +172,7 @@ class CacheManager extends Logging { case data if data.plan.find(lookupAndRefresh(_, fs, qualifiedPath)).isDefined => val dataIndex = cachedData.indexWhere(cd => data.plan.sameResult(cd.plan)) if (dataIndex >= 0) { - data.cachedRepresentation.unpersist() + data.cachedRepresentation.cachedColumnBuffers.unpersist(blocking = true) cachedData.remove(dataIndex) } sparkSession.sharedState.cacheManager.cacheQuery(Dataset.ofRows(sparkSession, data.plan)) 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 14024d6c1055..72bd20418830 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 @@ -205,7 +205,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return false; } - ${classOf[CachedBatch].getName} batch = (${classOf[CachedBatch].getName}) input.next(); + ${classOf[CachedBatchBytes].getName} batch = + (${classOf[CachedBatchBytes].getName}) input.next(); currentRow = 0; numRowsInBatch = batch.numRows(); for (int i = 0; i < columnIndexes.length; i ++) { 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 a281d5294a6f..b94ae9ecd3a4 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 @@ -46,11 +46,10 @@ object InMemoryRelation { /** - * CachedBatch is a cached batch of rows. + * Container for a physical plan that should be cached in memory. * - * @param numRows The total number of rows in this batch - * @param buffers The buffers for serialized columns - * @param stats The stat of columns + * This batches the rows from that plan into [[CachedBatch]]es that are later consumed by + * [[InMemoryTableScanExec]]. */ private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) @@ -66,9 +65,13 @@ case class InMemoryRelation( val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { - // Fallback to using ColumnBuilders if the schema has non-primitive types - private[columnar] val useColumnBatches: Boolean = { + /** + * If true, store the input rows using [[CachedColumnarBatch]]es, which are generally faster. + * If false, store the input rows using [[CachedBatchBytes]]. + */ + private[columnar] val useColumnarBatches: Boolean = { val enabled = child.sqlContext.conf.getConf(SQLConf.CACHE_CODEGEN) + // Fallback to storing the rows as bytes if the schema has non-primitive types val supported = output.forall { a => GenerateColumnarBatch.isSupported(a.dataType) } enabled && supported } @@ -89,44 +92,33 @@ case class InMemoryRelation( } } - // If the cached column buffers were not passed in, we calculate them in the constructor. - // As in Spark, the actual work of caching is lazy. - if (useColumnBatches && _cachedColumnBatches == null) { - buildColumnarBatches() - } - - if (!useColumnBatches && _cachedColumnBuffers == null) { - buildBuffers() - } - - def unpersist(blocking: Boolean = true): Unit = { - if (_cachedColumnBuffers != null) { - _cachedColumnBuffers.unpersist(blocking) - _cachedColumnBuffers = null - } - if (_cachedColumnBatches != null) { - _cachedColumnBatches.unpersist(blocking) - _cachedColumnBatches = null - } - } - - def recache(): Unit = { - unpersist() - if (useColumnBatches) { - buildColumnarBatches() - } else { - buildBuffers() - } + /** + * Batch the input rows into [[CachedBatch]]es. + */ + private def buildColumnBuffers(): RDD[CachedBatch] = { + val buffers = + if (useColumnarBatches) { + buildColumnarBatches() + } else { + buildColumnBytes() + } + buffers.setName( + tableName.map { n => s"In-memory table $n" } + .getOrElse(StringUtils.abbreviate(child.toString, 1024))) + buffers.asInstanceOf[RDD[CachedBatch]] } /** - * Batch the input rows using [[ColumnBuilder]]s. + * Batch the input rows into [[CachedBatchBytes]] built using [[ColumnBuilder]]s. + * + * This handles complex types and compression, but is more expensive than + * [[buildColumnarBatches]], which generates code to build the buffers. */ - private def buildBuffers(): Unit = { + private def buildColumnBytes(): RDD[CachedBatchBytes] = { val output = child.output - val cached = child.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { + child.execute().mapPartitionsInternal { rowIterator => + new Iterator[CachedBatchBytes] { + def next(): CachedBatchBytes = { val columnBuilders = output.map { attribute => ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray @@ -161,7 +153,7 @@ case class InMemoryRelation( val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) .flatMap(_.values)) - CachedBatch(rowCount, columnBuilders.map { builder => + CachedBatchBytes(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) } @@ -169,26 +161,30 @@ case class InMemoryRelation( def hasNext: Boolean = rowIterator.hasNext } }.persist(storageLevel) - - cached.setName( - tableName.map(n => s"In-memory table $n") - .getOrElse(StringUtils.abbreviate(child.toString, 1024))) - _cachedColumnBuffers = cached } /** * Batch the input rows using [[ColumnarBatch]]es. - * This provides a faster implementation of in-memory scan. + * + * Compared with [[buildColumnBytes]], this provides a faster implementation of memory + * scan because both the read path and the write path are generated. This only supports + * basic primitive types and does not compress data, however. */ - private def buildColumnarBatches(): Unit = { + private def buildColumnarBatches(): RDD[CachedColumnarBatch] = { val schema = StructType.fromAttributes(child.output) - val cached = child.execute().mapPartitionsInternal { rowIterator => - new GenerateColumnarBatch(schema, batchSize).generate(rowIterator) + child.execute().mapPartitionsInternal { rows => + new GenerateColumnarBatch(schema, batchSize).generate(rows).map { b => + CachedColumnarBatch(b) + } }.persist(storageLevel) - cached.setName( - tableName.map(n => s"In-memory table $n") - .getOrElse(StringUtils.abbreviate(child.toString, 1024))) - _cachedColumnBatches = cached + } + + def recache(): Unit = { + if (_cachedColumnBuffers != null) { + _cachedColumnBuffers.unpersist() + _cachedColumnBuffers = null + } + _cachedColumnBuffers = buildColumnBuffers() } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { @@ -209,9 +205,16 @@ case class InMemoryRelation( batchStats).asInstanceOf[this.type] } - def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers - def cachedColumnBatches: RDD[ColumnarBatch] = _cachedColumnBatches + /** + * Return lazily cached batches of rows in the original plan. + */ + def cachedColumnBuffers: RDD[CachedBatch] = { + if (_cachedColumnBuffers == null) { + _cachedColumnBuffers = buildColumnBuffers() + } + _cachedColumnBuffers + } override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, _cachedColumnBatches, batchStats) + Seq(_cachedColumnBuffers, batchStats) } 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 ec0a5fedcef1..986b9b218e06 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 @@ -33,13 +33,16 @@ case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode with ColumnarBatchScan { - override val supportCodegen: Boolean = relation.useColumnBatches + override val supportCodegen: Boolean = relation.useColumnarBatches override def inputRDDs(): Seq[RDD[InternalRow]] = { - if (relation.useColumnBatches) { - // HACK ALERT: This is actually an RDD[ColumnarBatch]. + if (relation.useColumnarBatches) { + // HACK ALERT: This is actually an RDD[CachedColumnarBatch]. // We're taking advantage of Scala's type erasure here to pass these batches along. - Seq(relation.cachedColumnBatches.asInstanceOf[RDD[InternalRow]]) + Seq(relation.cachedColumnBuffers + .asInstanceOf[RDD[CachedColumnarBatch]] + .map(_.columnarBatch) + .asInstanceOf[RDD[InternalRow]]) } else { Seq() } @@ -129,7 +132,7 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning protected override def doExecute(): RDD[InternalRow] = { - assert(!relation.useColumnBatches) + assert(!relation.useColumnarBatches) assert(relation.cachedColumnBuffers != null) val numOutputRows = longMetric("numOutputRows") @@ -143,7 +146,7 @@ case class InMemoryTableScanExec( val schema = relation.partitionStatistics.schema val schemaIndex = schema.zipWithIndex val relOutput: AttributeSeq = relation.output - val buffers = relation.cachedColumnBuffers + val buffers = relation.cachedColumnBuffers.asInstanceOf[RDD[CachedBatchBytes]] buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => val partitionFilter = newPredicate( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 3bb829069007..51ee0312f5fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -420,30 +420,13 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { storageLevel = MEMORY_ONLY, child = sparkPlan, tableName = None) - if (!useComplexSchema) { - assert(inMemoryRelation.useColumnBatches == useColumnBatches) - assert((inMemoryRelation.cachedColumnBatches != null) == useColumnBatches) - assert((inMemoryRelation.cachedColumnBuffers == null) == useColumnBatches) - } else { - // Fallback on using non-code-gen'ed column builders if schema is complex - assert(!inMemoryRelation.useColumnBatches) - assert(inMemoryRelation.cachedColumnBatches == null) - assert(inMemoryRelation.cachedColumnBuffers != null) + assert(inMemoryRelation.useColumnarBatches == useColumnBatches && !useComplexSchema) + assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == MEMORY_ONLY) + inMemoryRelation.cachedColumnBuffers.collect().head match { + case _: CachedColumnarBatch => assert(useColumnBatches && !useComplexSchema) + case _: CachedBatchBytes => assert(!useColumnBatches || useComplexSchema) + case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") } - // Test unpersist and recaching - def rdd: RDD[_] = - if (useColumnBatches && !useComplexSchema) { - inMemoryRelation.cachedColumnBatches - } else { - inMemoryRelation.cachedColumnBuffers - } - assert(rdd != null) - assert(rdd.getStorageLevel == MEMORY_ONLY) - inMemoryRelation.recache() - assert(rdd.getStorageLevel == MEMORY_ONLY) - inMemoryRelation.unpersist(blocking = true) - assert(inMemoryRelation.cachedColumnBatches == null) - assert(inMemoryRelation.cachedColumnBuffers == null) } } From df6c1f24c8c1d627acde4952bfff6354628908ed Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 24 Jun 2016 14:53:10 -0700 Subject: [PATCH 19/40] Add end-to-end benchmark, including write path --- .../execution/benchmark/CacheBenchmark.scala | 74 +++++++++++++------ 1 file changed, 53 insertions(+), 21 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala index b337c749d08d..b904d09a4cc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala @@ -25,6 +25,38 @@ import org.apache.spark.util.Benchmark class CacheBenchmark extends BenchmarkBase { + test("cache with randomized keys - end-to-end") { + benchmarkRandomizedKeys(size = 20 << 18, readPathOnly = false) + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ---------------------------------------------------------------------------------------------- + cache = F 641 / 667 8.2 122.2 1.0X + cache = T columnar_batches = F compress = F 1696 / 1833 3.1 323.6 0.4X + cache = T columnar_batches = F compress = T 7517 / 7748 0.7 1433.8 0.1X + cache = T columnar_batches = T 1023 / 1102 5.1 195.0 0.6X + */ + } + + test("cache with randomized keys - read path only") { + benchmarkRandomizedKeys(size = 20 << 21, readPathOnly = true) + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ----------------------------------------------------------------------------------------------- + cache = F 890 / 920 47.1 21.2 1.0X + cache = T columnar_batches = F compress = F 1950 / 1978 21.5 46.5 0.5X + cache = T columnar_batches = F compress = T 1893 / 1927 22.2 45.1 0.5X + cache = T columnar_batches = T 540 / 544 77.7 12.9 1.6X + */ + } + /** * Call collect on a [[DataFrame]] after deleting all existing temporary files. * This also checks whether the collected result matches the expected answer. @@ -41,17 +73,18 @@ class CacheBenchmark extends BenchmarkBase { } } - /* + /** + * Benchmark caching randomized keys created from a range. + * * NOTE: When running this benchmark, you will get a lot of WARN logs complaining that the * shuffle files do not exist. This is intentional; we delete the shuffle files manually * after every call to `collect` to avoid the next run to reuse shuffle files written by * the previous run. */ - test("cache with randomized keys") { - val N = 20 << 21 + private def benchmarkRandomizedKeys(size: Int, readPathOnly: Boolean): Unit = { val numIters = 10 - val benchmark = new Benchmark("Cache random keys", N) - sparkSession.range(N) + val benchmark = new Benchmark("Cache random keys", size) + sparkSession.range(size) .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") val query = "select count(k), count(id) from test" @@ -63,16 +96,27 @@ class CacheBenchmark extends BenchmarkBase { def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { val ds = sparkSession.sql(query) val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } - val prepare = () => { + def prepare(): Unit = { params.foreach { case (k, v) => sparkSession.conf.set(k, v) } if (cache) { sparkSession.catalog.cacheTable("test") } - collect(ds, expectedAnswer) + if (readPathOnly) { + collect(ds, expectedAnswer) + } } - val cleanup = () => { + def cleanup(): Unit = { defaults.foreach { case (k, v) => sparkSession.conf.set(k, v) } sparkSession.catalog.clearCache() } - benchmark.addCase(name, numIters, prepare, cleanup) { _ => collect(ds, expectedAnswer) } + benchmark.addCase(name, numIters, prepare, cleanup) { _ => + if (readPathOnly) { + collect(ds, expectedAnswer) + } else { + // also benchmark the time it takes to build the column buffers + val ds2 = sparkSession.sql(query) + collect(ds2, expectedAnswer) + collect(ds2, expectedAnswer) + } + } } // All of these are codegen = T hashmap = T @@ -97,18 +141,6 @@ class CacheBenchmark extends BenchmarkBase { SQLConf.CACHE_CODEGEN.key -> "true" )) benchmark.run() - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ----------------------------------------------------------------------------------------------- - cache = F 890 / 920 47.1 21.2 1.0X - cache = T columnar_batches = F compress = F 1950 / 1978 21.5 46.5 0.5X - cache = T columnar_batches = F compress = T 1893 / 1927 22.2 45.1 0.5X - cache = T columnar_batches = T 540 / 544 77.7 12.9 1.6X - */ } } From aa22b1669292314bad7daf15c04418ac3ece31ab Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 1 Sep 2016 11:47:25 +0900 Subject: [PATCH 20/40] merge with master --- .../sql/execution/DataSourceScanExec.scala | 1 + .../execution/columnar/InMemoryRelation.scala | 28 +++++++++++++++++-- .../columnar/InMemoryTableScanExec.scala | 11 +------- 3 files changed, 27 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 39b010efec7b..6276ebc36401 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer import org.apache.commons.lang3.StringUtils + import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} import org.apache.spark.rdd.RDD 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 b94ae9ecd3a4..8b23cebf143b 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 @@ -34,6 +34,31 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.LongAccumulator +/** + * An abstract representation of a cached batch of rows. + */ +private[columnar] trait CachedBatch + + +/** + * A cached batch of rows stored as a list of byte arrays, one for each column. + * + * @param numRows The total number of rows in this batch + * @param buffers The serialized buffers for serialized columns + * @param stats The stat of columns + */ +private[columnar] case class CachedBatchBytes( + numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + extends CachedBatch + + +/** + * A cached batch of rows stored as a [[ColumnarBatch]]. + */ +private[columnar] case class CachedColumnarBatch(columnarBatch: ColumnarBatch) + extends CachedBatch + + object InMemoryRelation { def apply( useCompression: Boolean, @@ -51,9 +76,6 @@ object InMemoryRelation { * This batches the rows from that plan into [[CachedBatch]]es that are later consumed by * [[InMemoryTableScanExec]]. */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) - case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, 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 986b9b218e06..88e03074f44a 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 @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType @@ -189,16 +190,6 @@ case class InMemoryTableScanExec( batch } - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulators && columnarIterator.hasNext) { - readPartitions.add(1) - } - val columnTypes = requestedColumnDataTypes.map { case udt: UserDefinedType[_] => udt.sqlType case other => other From e954ac443fbcc09d39b6ac156527f340621cee79 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 24 Sep 2016 03:26:11 +0900 Subject: [PATCH 21/40] support all of data types --- .../org/apache/spark/memory/MemoryMode.java | 3 +- .../execution/vectorized/ColumnVector.java | 150 ++++++- .../execution/vectorized/ColumnarBatch.java | 3 +- .../vectorized/OnHeapUnsafeColumnVector.java | 383 ++++++++++++++++++ .../sql/execution/ColumnarBatchScan.scala | 5 +- .../columnar/GenerateColumnarBatch.scala | 112 +++-- .../execution/columnar/InMemoryRelation.scala | 26 +- .../columnar/InMemoryTableScanExec.scala | 7 +- .../columnar/InMemoryColumnarQuerySuite.scala | 114 +++++- .../vectorized/ColumnarBatchSuite.scala | 24 +- 10 files changed, 751 insertions(+), 76 deletions(-) create mode 100755 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java diff --git a/core/src/main/java/org/apache/spark/memory/MemoryMode.java b/core/src/main/java/org/apache/spark/memory/MemoryMode.java index 3a5e72d8aaec..ca26a77ee9f6 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryMode.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryMode.java @@ -22,5 +22,6 @@ @Private public enum MemoryMode { ON_HEAP, - OFF_HEAP + OFF_HEAP, + ON_HEAP_UNSAFE } 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 354c878aca00..9c7d4c397dfd 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 @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.io.Serializable; import java.math.BigDecimal; import java.math.BigInteger; @@ -25,10 +26,15 @@ import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.MutableRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData; +import org.apache.spark.sql.catalyst.expressions.UnsafeMapData; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -57,7 +63,9 @@ * * ColumnVectors are intended to be reused. */ -public abstract class ColumnVector implements AutoCloseable { +public abstract class ColumnVector implements AutoCloseable, Serializable { + ColumnVector() { } + /** * Allocates a column to store elements of `type` on or off heap. * Capacity is the initial capacity of the vector and it will grow as necessary. Capacity is @@ -66,6 +74,8 @@ public abstract class ColumnVector implements AutoCloseable { public static ColumnVector allocate(int capacity, DataType type, MemoryMode mode) { if (mode == MemoryMode.OFF_HEAP) { return new OffHeapColumnVector(capacity, type); + } else if (mode == MemoryMode.ON_HEAP_UNSAFE) { + return new OnHeapUnsafeColumnVector(capacity, type); } else { return new OnHeapColumnVector(capacity, type); } @@ -548,18 +558,69 @@ public ColumnarBatch.Row getStruct(int rowId) { * Returns a utility object to get structs. * provided to keep API compatibility with InternalRow for code generation */ - public ColumnarBatch.Row getStruct(int rowId, int size) { - resultStruct.rowId = rowId; - return resultStruct; + public MutableRow getStruct(int rowId, int size) { + if (!unsafeDirectCopy) { + resultStruct.rowId = rowId; + return resultStruct; + } + resultArray.data.loadBytes(resultArray); + int offset = getArrayOffset(rowId); + int length = getArrayLength(rowId); + UnsafeRow map = new UnsafeRow(size); + map.pointTo(resultArray.byteArray, Platform.BYTE_ARRAY_OFFSET + offset, length); + return map; + } + + public int putStruct(int rowId, InternalRow row) { + if (!unsafeDirectCopy) { + throw new UnsupportedOperationException(); + } + assert(row instanceof UnsafeRow); + UnsafeRow unsafeRow = (UnsafeRow)row; + byte[] value = (byte[])unsafeRow.getBaseObject(); + long offset = unsafeRow.getBaseOffset() - Platform.BYTE_ARRAY_OFFSET; + int length = unsafeRow.getSizeInBytes(); + if (offset > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot put this map to ColumnVector as " + + "it's too big."); + } + putByteArray(rowId, value, (int)offset, length); + return length; } /** * Returns the array at rowid. */ - public final Array getArray(int rowId) { - resultArray.length = getArrayLength(rowId); - resultArray.offset = getArrayOffset(rowId); - return resultArray; + public final ArrayData getArray(int rowId) { + if (unsafeDirectCopy) { + resultArray.data.loadBytes(resultArray); // update resultArray.byteData + int offset = getArrayOffset(rowId); + int length = getArrayLength(rowId); + UnsafeArrayData array = new UnsafeArrayData(); + array.pointTo(resultArray.byteArray, Platform.BYTE_ARRAY_OFFSET + offset, length); + return array; + } else { + resultArray.length = getArrayLength(rowId); + resultArray.offset = getArrayOffset(rowId); + return resultArray; + } + } + + public final int putArray(int rowId, ArrayData array) { + if (!unsafeDirectCopy) { + throw new UnsupportedOperationException(); + } + assert(array instanceof UnsafeArrayData); + UnsafeArrayData unsafeArray = (UnsafeArrayData)array; + byte[] value = (byte[])unsafeArray.getBaseObject(); + long offset = unsafeArray.getBaseOffset() - Platform.BYTE_ARRAY_OFFSET; + int length = unsafeArray.getSizeInBytes(); + if (offset > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot put this array to ColumnVector as " + + "it's too big."); + } + putByteArray(rowId, value, (int)offset, length); + return length; } /** @@ -579,7 +640,9 @@ public final int putByteArray(int rowId, byte[] value) { * Returns the value for rowId. */ private Array getByteArray(int rowId) { - Array array = getArray(rowId); + resultArray.length = getArrayLength(rowId); + resultArray.offset = getArrayOffset(rowId); + Array array = resultArray; array.data.loadBytes(array); return array; } @@ -587,8 +650,33 @@ private Array getByteArray(int rowId) { /** * Returns the value for rowId. */ - public MapData getMap(int ordinal) { - throw new UnsupportedOperationException(); + public MapData getMap(int rowId) { + if (!unsafeDirectCopy) { + throw new UnsupportedOperationException(); + } + resultArray.data.loadBytes(resultArray); + int offset = getArrayOffset(rowId); + int length = getArrayLength(rowId); + UnsafeMapData map = new UnsafeMapData(); + map.pointTo(resultArray.byteArray, Platform.BYTE_ARRAY_OFFSET + offset, length); + return map; + } + + public int putMap(int rowId, MapData map) { + if (!unsafeDirectCopy) { + throw new UnsupportedOperationException(); + } + assert(map instanceof UnsafeMapData); + UnsafeMapData unsafeMap = (UnsafeMapData)map; + byte[] value = (byte[])unsafeMap.getBaseObject(); + long offset = unsafeMap.getBaseOffset() - Platform.BYTE_ARRAY_OFFSET; + int length = unsafeMap.getSizeInBytes(); + if (offset > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot put this map to ColumnVector as " + + "it's too big."); + } + putByteArray(rowId, value, (int)offset, length); + return length; } /** @@ -609,14 +697,18 @@ public final Decimal getDecimal(int rowId, int precision, int scale) { } - public final void putDecimal(int rowId, Decimal value, int precision) { + public final int putDecimal(int rowId, Decimal value, int precision) { if (precision <= Decimal.MAX_INT_DIGITS()) { putInt(rowId, (int) value.toUnscaledLong()); + return 4; } else if (precision <= Decimal.MAX_LONG_DIGITS()) { putLong(rowId, value.toUnscaledLong()); + return 8; } else { BigInteger bigInteger = value.toJavaBigDecimal().unscaledValue(); - putByteArray(rowId, bigInteger.toByteArray()); + byte[] array = bigInteger.toByteArray(); + putByteArray(rowId, array); + return array.length; } } @@ -633,6 +725,13 @@ public final UTF8String getUTF8String(int rowId) { } } + public final int putUTF8String(int rowId, UTF8String string) { + assert(dictionary == null); + byte[] array = string.getBytes(); + putByteArray(rowId, array); + return array.length; + } + /** * Returns the byte array for rowId. */ @@ -648,6 +747,11 @@ public final byte[] getBinary(int rowId) { } } + public final int putBinary(int rowId, byte[] bytes) { + putByteArray(rowId, bytes); + return bytes.length; + } + /** * Append APIs. These APIs all behave similarly and will append data to the current vector. It * is not valid to mix the put and append APIs. The append APIs are slower and should only be @@ -894,10 +998,12 @@ public final int appendStruct(boolean isNull) { @VisibleForTesting protected int MAX_CAPACITY = Integer.MAX_VALUE; + protected boolean unsafeDirectCopy; + /** * Data type for this column. */ - protected final DataType type; + protected DataType type; /** * Number of nulls in this column. This is an optimization for the reader, to skip NULL checks. @@ -929,17 +1035,17 @@ public final int appendStruct(boolean isNull) { /** * If this is a nested type (array or struct), the column for the child data. */ - protected final ColumnVector[] childColumns; + protected ColumnVector[] childColumns; /** * Reusable Array holder for getArray(). */ - protected final Array resultArray; + protected Array resultArray; /** * Reusable Struct holder for getStruct(). */ - protected final ColumnarBatch.Row resultStruct; + protected ColumnarBatch.Row resultStruct; /** * The Dictionary for this column. @@ -991,14 +1097,20 @@ public ColumnVector getDictionaryIds() { * type. */ protected ColumnVector(int capacity, DataType type, MemoryMode memMode) { + this(capacity, type, memMode, false); + } + + protected ColumnVector(int capacity, DataType type, MemoryMode memMode, boolean unsafeDirectCopy) { this.capacity = capacity; this.type = type; + this.unsafeDirectCopy = unsafeDirectCopy; if (type instanceof ArrayType || type instanceof BinaryType || type instanceof StringType - || DecimalType.isByteArrayDecimalType(type)) { + || DecimalType.isByteArrayDecimalType(type) + || unsafeDirectCopy && (type instanceof MapType || type instanceof StructType)) { DataType childType; int childCapacity = capacity; - if (type instanceof ArrayType) { + if (!unsafeDirectCopy && type instanceof ArrayType) { childType = ((ArrayType)type).elementType(); } else { childType = DataTypes.ByteType; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index a6ce4c2edc23..d9ec7145622a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.io.Serializable; import java.math.BigDecimal; import java.util.*; @@ -43,7 +44,7 @@ * - There are many TODOs for the existing APIs. They should throw a not implemented exception. * - Compaction: The batch and columns should be able to compact based on a selection vector. */ -public final class ColumnarBatch { +public final class ColumnarBatch implements Serializable { private static final int DEFAULT_BATCH_SIZE = 4 * 1024; private static MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java new file mode 100755 index 000000000000..f49f490395d4 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java @@ -0,0 +1,383 @@ +/* + * 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.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; + +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.Platform; + +/** + * A column backed by an in memory JVM array. But, all of data types are stored into byte[]. + * This stores the NULLs as a byte per value and a java array for the values. + */ +public final class OnHeapUnsafeColumnVector extends ColumnVector implements Serializable { + + // 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 all types + private byte[] data; + + // Only set if type is Array. + private int[] arrayLengths; + private int[] arrayOffsets; + + OnHeapUnsafeColumnVector() { } + + protected OnHeapUnsafeColumnVector(int capacity, DataType type) { + super(capacity, type, MemoryMode.ON_HEAP, true); + reserveInternal(capacity); + reset(); + } + + @Override + public long valuesNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + @Override + public long nullsNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + + @Override + public void close() { + } + + // + // APIs dealing with nulls + // + + @Override + public void putNotNull(int rowId) { + nulls[rowId] = (byte)0; + } + + @Override + public void putNull(int rowId) { + nulls[rowId] = (byte)1; + ++numNulls; + anyNullsSet = true; + } + + @Override + public void putNulls(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + @Override + public void putNotNulls(int rowId, int count) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isNullAt(int rowId) { + return nulls[rowId] == 1; + } + + // + // APIs dealing with Booleans + // + + @Override + public void putBoolean(int rowId, boolean value) { + Platform.putBoolean(data, Platform.BYTE_ARRAY_OFFSET + rowId, value); + } + + @Override + public void putBooleans(int rowId, int count, boolean value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean getBoolean(int rowId) { + return Platform.getBoolean(data, Platform.BYTE_ARRAY_OFFSET + rowId); + } + + // + + // + // APIs dealing with Bytes + // + + @Override + public void putByte(int rowId, byte value) { + Platform.putByte(data, Platform.BYTE_ARRAY_OFFSET + rowId, value); + } + + @Override + public void putBytes(int rowId, int count, byte value) { + for (int i = 0; i < count; ++i) { + data[i + rowId] = value; + } + } + + @Override + public void putBytes(int rowId, int count, byte[] src, int srcIndex) { + System.arraycopy(src, srcIndex, data, rowId, count); + } + + @Override + public byte getByte(int rowId) { + return Platform.getByte(data, Platform.BYTE_ARRAY_OFFSET + rowId); + } + + // + // APIs dealing with Shorts + // + + @Override + public void putShort(int rowId, short value) { + Platform.putShort(data, Platform.BYTE_ARRAY_OFFSET + rowId * 2, value); + } + + @Override + public void putShorts(int rowId, int count, short value) { + throw new UnsupportedOperationException(); + } + + @Override + public void putShorts(int rowId, int count, short[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + return Platform.getShort(data, Platform.BYTE_ARRAY_OFFSET + rowId * 2); + } + + + // + // APIs dealing with Ints + // + + @Override + public void putInt(int rowId, int value) { + Platform.putInt(data, Platform.BYTE_ARRAY_OFFSET + rowId * 4, value); + } + + @Override + public void putInts(int rowId, int count, int value) { + throw new UnsupportedOperationException(); + } + + @Override + public void putInts(int rowId, int count, int[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + return Platform.getInt(data, Platform.BYTE_ARRAY_OFFSET + rowId * 4); + } + + /** + * Returns the dictionary Id for rowId. + * This should only be called when the ColumnVector is dictionaryIds. + * We have this separate method for dictionaryIds as per SPARK-16928. + */ + public int getDictId(int rowId) { throw new UnsupportedOperationException(); } + + // + // APIs dealing with Longs + // + + @Override + public void putLong(int rowId, long value) { + Platform.putLong(data, Platform.BYTE_ARRAY_OFFSET + rowId * 8, value); + } + + @Override + public void putLongs(int rowId, int count, long value) { + throw new UnsupportedOperationException(); + } + + @Override + public void putLongs(int rowId, int count, long[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + return Platform.getLong(data, Platform.BYTE_ARRAY_OFFSET + rowId * 8); + } + + // + // APIs dealing with floats + // + + @Override + public void putFloat(int rowId, float value) { + Platform.putFloat(data, Platform.BYTE_ARRAY_OFFSET + rowId * 4, value); + } + + @Override + public void putFloats(int rowId, int count, float value) { + throw new UnsupportedOperationException(); + } + + @Override + public void putFloats(int rowId, int count, float[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public void putFloats(int rowId, int count, byte[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + return Platform.getFloat(data, Platform.BYTE_ARRAY_OFFSET + rowId * 4); + } + + // + // APIs dealing with doubles + // + + @Override + public void putDouble(int rowId, double value) { + Platform.putDouble(data, Platform.BYTE_ARRAY_OFFSET + rowId * 8, value); + } + + @Override + public void putDoubles(int rowId, int count, double value) { + throw new UnsupportedOperationException(); + } + + @Override + public void putDoubles(int rowId, int count, double[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + return Platform.getDouble(data, Platform.BYTE_ARRAY_OFFSET + rowId * 8); + } + + // + // APIs dealing with Arrays + // + + @Override + public int getArrayLength(int rowId) { + return arrayLengths[rowId]; + } + @Override + public int getArrayOffset(int rowId) { + return arrayOffsets[rowId]; + } + + @Override + public void putArray(int rowId, int offset, int length) { + arrayOffsets[rowId] = offset; + arrayLengths[rowId] = length; + } + + @Override + public void loadBytes(ColumnVector.Array array) { + array.byteArray = data; + array.byteArrayOffset = array.offset; + } + + // + // APIs dealing with Byte Arrays + // + + @Override + public int putByteArray(int rowId, byte[] value, int offset, int length) { + int result = arrayData().appendBytes(length, value, offset); + arrayOffsets[rowId] = result; + arrayLengths[rowId] = length; + return result; + } + + // Spilt this function out since it is the slow path. + @Override + protected void reserveInternal(int newCapacity) { + int factor = 0; + if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { + int[] newLengths = new int[newCapacity]; + int[] newOffsets = new int[newCapacity]; + if (this.arrayLengths != null) { + System.arraycopy(this.arrayLengths, 0, newLengths, 0, elementsAppended); + System.arraycopy(this.arrayOffsets, 0, newOffsets, 0, elementsAppended); + } + arrayLengths = newLengths; + arrayOffsets = newOffsets; + factor = -1; + } else if (resultStruct != null || type instanceof NullType) { + // Nothing to store. + factor = -1; + } else if (type instanceof BooleanType) { + factor = 1; + } else if (type instanceof ByteType) { + factor = 1; + } else if (type instanceof ShortType) { + factor = 2; + } else if (type instanceof IntegerType || type instanceof DateType || + DecimalType.is32BitDecimalType(type)) { + factor = 4; + } else if (type instanceof LongType || type instanceof TimestampType || + DecimalType.is64BitDecimalType(type)) { + factor = 8; + } else if (type instanceof FloatType) { + factor = 4; + } else if (type instanceof DoubleType) { + factor = 8; + } + if (factor > 0) { + if (data == null || capacity < newCapacity) { + byte[] newData = new byte[newCapacity * factor]; + if (data != null) + Platform.copyMemory(data, Platform.BYTE_ARRAY_OFFSET, + newData, Platform.BYTE_ARRAY_OFFSET, elementsAppended * factor); + data = newData; + } + } else if (factor == 0) { + throw new RuntimeException("Unhandled " + type); + } + + byte[] newNulls = new byte[newCapacity]; + if (nulls != null) System.arraycopy(nulls, 0, newNulls, 0, elementsAppended); + nulls = newNulls; + + capacity = newCapacity; + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 04fba17be4bf..01fc51bbc722 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -33,6 +33,8 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { val inMemoryTableScan: InMemoryTableScanExec = null + val columnIndexes: Array[Int] = null + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) @@ -89,7 +91,8 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) val columnAssigns = colVars.zipWithIndex.map { case (name, i) => ctx.addMutableState(columnVectorClz, name, s"$name = null;") - s"$name = $batch.column($i);" + val index = if (columnIndexes == null) i else columnIndexes(i) + s"$name = $batch.column($index);" } val nextBatch = ctx.freshName("nextBatch") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 1becca162fca..469192b12518 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter} -import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, CodegenContext} import org.apache.spark.sql.execution.vectorized.ColumnarBatch import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel /** @@ -36,7 +36,8 @@ abstract class ColumnarBatchIterator extends Iterator[ColumnarBatch] */ class GenerateColumnarBatch( schema: StructType, - batchSize: Int) + batchSize: Int, + storageLevel: StorageLevel) extends CodeGenerator[Iterator[InternalRow], Iterator[ColumnarBatch]] { protected def canonicalize(in: Iterator[InternalRow]): Iterator[InternalRow] = in @@ -58,19 +59,21 @@ class GenerateColumnarBatch( val schemaVar = ctx.addReferenceObj("schema", schema, classOf[StructType].getName) val maxNumBytes = ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE // Code to populate column vectors with the values of the input rows - val populateColumnVectorsCode = schema.fields.zipWithIndex.map { case (field, i) => - val typeName = GenerateColumnarBatch.typeToName(field.dataType) - val put = "put" + typeName.capitalize - val get = "get" + typeName.capitalize - s""" - $batchVar.column($i).$put($rowNumVar, row.$get($i)); - $numBytesVar += ${field.dataType.defaultSize}; - """.trim + val colVars = schema.fields.indices.map(i => ctx.freshName("colInstance" + i)) + val columnInstanceCode = colVars.zipWithIndex.map { case (name, i) => + s"ColumnVector $name = $batchVar.column($i);" }.mkString("\n") + + val populateColumnVectorsCode = (schema.fields zip colVars).zipWithIndex.map { + case ((field, colVar), i) => + GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, + colVar, "row", rowNumVar, i, numBytesVar) + }.mkString("") val code = s""" import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.vectorized.ColumnarBatch; + import org.apache.spark.sql.execution.vectorized.ColumnVector; public GeneratedColumnarBatchIterator generate(Object[] references) { return new GeneratedColumnarBatchIterator(references); @@ -91,7 +94,8 @@ class GenerateColumnarBatch( @Override public ColumnarBatch next() { ColumnarBatch $batchVar = - ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP, $batchSize); + ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP_UNSAFE, $batchSize); + $columnInstanceCode int $rowNumVar = 0; long $numBytesVar = 0; while ($rowIterVar.hasNext() && $rowNumVar < $batchSize && $numBytesVar < $maxNumBytes) { @@ -115,20 +119,74 @@ class GenerateColumnarBatch( private[columnar] object GenerateColumnarBatch { - private val typeToName = Map[DataType, String]( - BooleanType -> "boolean", - ByteType -> "byte", - ShortType -> "short", - IntegerType -> "int", - LongType -> "long", - FloatType -> "float", - DoubleType -> "double") - - /** - * Whether [[ColumnarBatch]]-based caching is supported for the given data type - */ - def isSupported(dataType: DataType): Boolean = { - typeToName.contains(dataType) + private val typeToName = Map[AbstractDataType, String]( + BooleanType -> "boolean", + ByteType -> "byte", + ShortType -> "short", + IntegerType -> "int", + LongType -> "long", + FloatType -> "float", + DoubleType -> "double", + DateType -> "int", + TimestampType -> "long", + StringType -> "UTF8String", + BinaryType -> "Binary" + ) + + def putColumnCode(ctx: CodegenContext, dt: DataType, nullable: Boolean, colVar: String, + rowVar: String, rowNumVar: String, colNum: Int, numBytesVar: String) : String = { + val body = dt match { + case t if ctx.isPrimitiveType(dt) => + val typeName = GenerateColumnarBatch.typeToName(dt) + val put = "put" + typeName.capitalize + val get = "get" + typeName.capitalize + s""" + |$colVar.$put($rowNumVar, $rowVar.$get($colNum)); + |$numBytesVar += ${dt.defaultSize}; + """.stripMargin + case StringType | BinaryType => + val typeName = GenerateColumnarBatch.typeToName(dt) + val put = "put" + typeName.capitalize + val get = "get" + typeName.capitalize + s"""$numBytesVar += $colVar.$put($rowNumVar, $rowVar.$get($colNum));""" + case NullType => + return s""" + |if ($rowVar.isNullAt($colNum)) { + | $colVar.putNull($rowNumVar); + |} else { + | $colVar.putNotNull($rowNumVar); + |} + |$numBytesVar += 1; + """.stripMargin + case dt: DecimalType => + val precision = dt.precision + val scale = dt.scale + s""" + $numBytesVar += $colVar.putDecimal($rowNumVar, + $rowVar.getDecimal($colNum, $precision, $scale), $precision); + """.stripMargin + case array: ArrayType => + s"""$numBytesVar += $colVar.putArray($rowNumVar, $rowVar.getArray($colNum));""" + case t: MapType => + s"""$numBytesVar += $colVar.putMap($rowNumVar, $rowVar.getMap($colNum));""" + case struct: StructType => + s""" + $numBytesVar += $colVar.putStruct($rowNumVar, + $rowVar.getStruct($colNum, ${struct.length})); + """.stripMargin + case _ => + throw new UnsupportedOperationException("Unsupported data type " + dt.simpleString); + } + if (nullable) { + s""" + |if ($rowVar.isNullAt($colNum)) { + | $colVar.putNull($rowNumVar); + |} else { + | $body + |} + """.stripMargin + } else { + body + } } - } 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 8b23cebf143b..efe1872f207e 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 @@ -92,10 +92,7 @@ case class InMemoryRelation( * If false, store the input rows using [[CachedBatchBytes]]. */ private[columnar] val useColumnarBatches: Boolean = { - val enabled = child.sqlContext.conf.getConf(SQLConf.CACHE_CODEGEN) - // Fallback to storing the rows as bytes if the schema has non-primitive types - val supported = output.forall { a => GenerateColumnarBatch.isSupported(a.dataType) } - enabled && supported + child.sqlContext.conf.getConf(SQLConf.CACHE_CODEGEN) } override protected def innerChildren: Seq[SparkPlan] = Seq(child) @@ -117,7 +114,7 @@ case class InMemoryRelation( /** * Batch the input rows into [[CachedBatch]]es. */ - private def buildColumnBuffers(): RDD[CachedBatch] = { + private def buildColumnBuffers: RDD[CachedBatch] = { val buffers = if (useColumnarBatches) { buildColumnarBatches() @@ -189,24 +186,29 @@ case class InMemoryRelation( * Batch the input rows using [[ColumnarBatch]]es. * * Compared with [[buildColumnBytes]], this provides a faster implementation of memory - * scan because both the read path and the write path are generated. This only supports - * basic primitive types and does not compress data, however. + * scan because both the read path and the write path are generated. + * However, this does not compress data for now */ private def buildColumnarBatches(): RDD[CachedColumnarBatch] = { val schema = StructType.fromAttributes(child.output) child.execute().mapPartitionsInternal { rows => - new GenerateColumnarBatch(schema, batchSize).generate(rows).map { b => - CachedColumnarBatch(b) - } + new GenerateColumnarBatch(schema, batchSize, storageLevel) + .generate(rows).map { columnarBatch => CachedColumnarBatch(columnarBatch) } }.persist(storageLevel) } + // If the cached column buffers were not passed in, we calculate them in the constructor. + // As in Spark, the actual work of caching is lazy. + if (_cachedColumnBuffers == null) { + _cachedColumnBuffers = buildColumnBuffers + } + def recache(): Unit = { if (_cachedColumnBuffers != null) { _cachedColumnBuffers.unpersist() _cachedColumnBuffers = null } - _cachedColumnBuffers = buildColumnBuffers() + _cachedColumnBuffers = buildColumnBuffers } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { @@ -232,7 +234,7 @@ case class InMemoryRelation( */ def cachedColumnBuffers: RDD[CachedBatch] = { if (_cachedColumnBuffers == null) { - _cachedColumnBuffers = buildColumnBuffers() + _cachedColumnBuffers = buildColumnBuffers } _cachedColumnBuffers } 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 88e03074f44a..1854acd83c48 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 @@ -34,6 +34,8 @@ case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode with ColumnarBatchScan { + override val columnIndexes = attributes.map(a => relation.output.indexOf(a)).toArray + override val supportCodegen: Boolean = relation.useColumnarBatches override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -52,7 +54,8 @@ case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) override def output: Seq[Attribute] = attributes @@ -134,7 +137,6 @@ case class InMemoryTableScanExec( protected override def doExecute(): RDD[InternalRow] = { assert(!relation.useColumnarBatches) - assert(relation.cachedColumnBuffers != null) val numOutputRows = longMetric("numOutputRows") if (enableAccumulators) { @@ -147,6 +149,7 @@ case class InMemoryTableScanExec( val schema = relation.partitionStatistics.schema val schemaIndex = schema.zipWithIndex val relOutput: AttributeSeq = relation.output + assert(relation.cachedColumnBuffers != null) val buffers = relation.cachedColumnBuffers.asInstanceOf[RDD[CachedBatchBytes]] buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 51ee0312f5fb..45bbb6cbe2e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -112,6 +112,112 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { tesNonPrimitiveType(false) } + def cachePrimitiveTest(data: DataFrame, dataType: String) { + data.createOrReplaceTempView(s"testData$dataType") + val useColumnBatches = true + withSQLConf(SQLConf.CACHE_CODEGEN.key -> useColumnBatches.toString) { + Seq(MEMORY_ONLY).map { storageLevel => + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val inMemoryRelation = InMemoryRelation(useCompression = false, 5, storageLevel, plan, None) + + assert(inMemoryRelation.useColumnarBatches == useColumnBatches) + assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == storageLevel) + inMemoryRelation.cachedColumnBuffers.collect().head match { + case _: CachedColumnarBatch => assert(useColumnBatches) + case _: CachedBatchBytes => assert(!useColumnBatches) + case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") + } + checkAnswer(inMemoryRelation, data.collect().toSeq) + } + } + } + + test("all data type w && w/o nullability") { + // all primitives + Seq(true, false).map { nullability => + val dataTypes = Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType) + val schema = StructType(dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, nullability) + }) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else if (i % 2 == 0) true else false, + if (nullability && i % 3 == 0) null else i.toByte, + if (nullability && i % 3 == 0) null else i.toShort, + if (nullability && i % 3 == 0) null else i.toInt, + if (nullability && i % 3 == 0) null else i.toLong, + if (nullability && i % 3 == 0) null else (i + 0.25).toFloat, + if (nullability && i % 3 == 0) null else (i + 0.75).toDouble + ))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "primitives") + } + + val schemaNull = StructType(Seq(StructField(s"col", NullType, true))) + val rddNull = spark.sparkContext.parallelize((1 to 10).map(i => Row(null))) + cachePrimitiveTest(spark.createDataFrame(rddNull, schemaNull), "Null") + + Seq(true, false).map { nullability => + val schema = StructType(Seq(StructField(s"col0", DateType, nullability), + StructField(s"col1", TimestampType, nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else new Date(i), + if (nullability && i % 3 == 0) null else new Timestamp(i * 1000000L) + ))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "DateTimestamp") + } + + Seq(true, false).map { nullability => + val schema = StructType(Seq(StructField(s"col", StringType, nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else s"str${i}: test cache."))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "String") + } + + Seq(true, false).map { nullability => + val schema = StructType(Seq(StructField(s"col0", DecimalType(25, 5), nullability), + StructField(s"col1", DecimalType(6, 5), nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else BigDecimal(Long.MaxValue.toString + ".12345"), + if (nullability && i % 3 == 0) null + else new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456") + ))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Decimal") + } + + Seq(true, false).map { nullability => + val schema = StructType(Seq(StructField(s"col", ArrayType(IntegerType), nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else (i * 100 to i * 100 + i).toArray))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Array") + } + + Seq(true, false).map { nullability => + val schema = StructType( + Seq(StructField(s"col", ArrayType(ArrayType(IntegerType)), nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null + else Array(Array(i, i + 1), Array(i * 100 + 1, i * 100, i * 100 + 2))))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "ArrayArray") + } + + Seq(true, false).map { nullability => + val schema = StructType( + Seq(StructField(s"col", MapType(StringType, IntegerType), nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else (i to i + i).map(j => s"key$j" -> j).toMap))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Map") + } + + Seq(true, false).map { nullability => + val struct = StructType(StructField("f1", FloatType, false) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val schema = StructType(Seq(StructField(s"col", struct, nullability))) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else Row((i + 0.25).toFloat, Seq(true, false, null))))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Struct") + } + } + test("simple columnar query") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) @@ -318,6 +424,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val length2 = 8117 val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) + + /* TODO: test GenerateColumnarBatch().generate() with many columns */ } test("SPARK-17549: cached table size should be correctly calculated") { @@ -420,11 +528,11 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { storageLevel = MEMORY_ONLY, child = sparkPlan, tableName = None) - assert(inMemoryRelation.useColumnarBatches == useColumnBatches && !useComplexSchema) + assert(inMemoryRelation.useColumnarBatches == useColumnBatches) assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == MEMORY_ONLY) inMemoryRelation.cachedColumnBuffers.collect().head match { - case _: CachedColumnarBatch => assert(useColumnBatches && !useComplexSchema) - case _: CachedBatchBytes => assert(!useColumnBatches || useComplexSchema) + case _: CachedColumnarBatch => assert(useColumnBatches) + case _: CachedBatchBytes => assert(!useColumnBatches) case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 8184d7d909f4..7882e1dbba3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -464,26 +464,30 @@ class ColumnarBatchSuite extends SparkFunSuite { column.putArray(2, 2, 0) column.putArray(3, 3, 3) - val a1 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(0)).asInstanceOf[Array[Int]] - val a2 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(1)).asInstanceOf[Array[Int]] - val a3 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(2)).asInstanceOf[Array[Int]] - val a4 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(3)).asInstanceOf[Array[Int]] + val a1 = ColumnVectorUtils.toPrimitiveJavaArray( + column.getArray(0).asInstanceOf[ColumnVector.Array]).asInstanceOf[Array[Int]] + val a2 = ColumnVectorUtils.toPrimitiveJavaArray( + column.getArray(1).asInstanceOf[ColumnVector.Array]).asInstanceOf[Array[Int]] + val a3 = ColumnVectorUtils.toPrimitiveJavaArray( + column.getArray(2).asInstanceOf[ColumnVector.Array]).asInstanceOf[Array[Int]] + val a4 = ColumnVectorUtils.toPrimitiveJavaArray( + column.getArray(3).asInstanceOf[ColumnVector.Array]).asInstanceOf[Array[Int]] assert(a1 === Array(0)) assert(a2 === Array(1, 2)) assert(a3 === Array.empty[Int]) assert(a4 === Array(3, 4, 5)) // Verify the ArrayData APIs - assert(column.getArray(0).length == 1) + assert(column.getArray(0).numElements() == 1) assert(column.getArray(0).getInt(0) == 0) - assert(column.getArray(1).length == 2) + assert(column.getArray(1).numElements() == 2) assert(column.getArray(1).getInt(0) == 1) assert(column.getArray(1).getInt(1) == 2) - assert(column.getArray(2).length == 0) + assert(column.getArray(2).numElements() == 0) - assert(column.getArray(3).length == 3) + assert(column.getArray(3).numElements() == 3) assert(column.getArray(3).getInt(0) == 3) assert(column.getArray(3).getInt(1) == 4) assert(column.getArray(3).getInt(2) == 5) @@ -496,8 +500,8 @@ class ColumnarBatchSuite extends SparkFunSuite { assert(data.capacity == array.length * 2) data.putInts(0, array.length, array, 0) column.putArray(0, 0, array.length) - assert(ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(0)).asInstanceOf[Array[Int]] - === array) + assert(ColumnVectorUtils.toPrimitiveJavaArray( + column.getArray(0).asInstanceOf[ColumnVector.Array]).asInstanceOf[Array[Int]] === array) }} } From fc1f6aa66c67e7d97ded9a9816ad0fc3c7439ca2 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 23 Sep 2016 14:59:40 -0400 Subject: [PATCH 22/40] rebase --- .../org/apache/spark/sql/execution/DataSourceScanExec.scala | 1 - .../spark/sql/execution/columnar/GenerateColumnarBatch.scala | 2 +- .../spark/sql/execution/benchmark/AggregateBenchmark.scala | 2 ++ 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 6276ebc36401..39b010efec7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer import org.apache.commons.lang3.StringUtils - import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} import org.apache.spark.rdd.RDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 469192b12518..999f1802a4b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, CodegenContext} +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.execution.vectorized.ColumnarBatch import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 5b8fda47b9ce..8a2993bdf4b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -158,6 +158,8 @@ class AggregateBenchmark extends BenchmarkBase { f() } + benchmark.run() + /* Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz From 6f66119bd28076ff605e6aaf0ed86927ba49d459 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Sep 2016 00:34:26 +0900 Subject: [PATCH 23/40] support UDT and column pruning --- .../sql/execution/ColumnarBatchScan.scala | 169 ++++++++++++++++++ .../sql/execution/columnar/ColumnStats.scala | 103 +++++++++++ .../columnar/GenerateColumnarBatch.scala | 81 +++++++-- .../execution/columnar/InMemoryRelation.scala | 8 +- .../columnar/InMemoryTableScanExec.scala | 34 +++- 5 files changed, 373 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 01fc51bbc722..47d91799c64f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -1,3 +1,4 @@ +<<<<<<< fc1f6aa66c67e7d97ded9a9816ad0fc3c7439ca2 /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -39,6 +40,9 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + lazy val enableScanStatistics: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + /** * Generate [[ColumnVector]] expressions for our parent to consume as rows. * This is called once per [[ColumnarBatch]]. @@ -80,6 +84,17 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { val scanTimeMetric = metricTerm(ctx, "scanTime") val scanTimeTotalNs = ctx.freshName("scanTime") ctx.addMutableState("long", scanTimeTotalNs, s"$scanTimeTotalNs = 0;") + val incReadBatches = if (!enableScanStatistics) "" else { + val readPartitions = ctx.addReferenceObj("readPartitions", inMemoryTableScan.readPartitions) + val readBatches = ctx.addReferenceObj("readBatches", inMemoryTableScan.readBatches) + ctx.addMutableState("int", "initializeInMemoryTableScanStatistics", + s""" + |$readPartitions.setValue(0); + |$readBatches.setValue(0); + |if ($input.hasNext()) { $readPartitions.add(1); } + """.stripMargin) + s"$readBatches.add(1);" + } val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" val batch = ctx.freshName("batch") @@ -102,6 +117,7 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { | long getBatchStart = System.nanoTime(); | if ($input.hasNext()) { | $batch = ($columnarBatchClz)$input.next(); + | $incReadBatches | $numOutputRows.add($batch.numRows()); | $idx = 0; | ${columnAssigns.mkString("", "\n", "\n")} @@ -134,3 +150,156 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { } } +======= +/* + * 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.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.types.DataType + + +/** + * Helper trait for abstracting scan functionality using + * [[org.apache.spark.sql.execution.vectorized.ColumnarBatch]]es. + */ +private[sql] trait ColumnarBatchScan extends CodegenSupport { + + val columnIndexes: Array[Int] = null + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + + val inMemoryTableScan: InMemoryTableScanExec = null + + lazy val enableScanStatistics: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. + * This is called once per [[ColumnarBatch]]. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = ctx.javaType(dataType) + val value = ctx.getValue(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { + s""" + boolean $isNullVar = $columnVar.isNullAt($ordinal); + $javaType $valueVar = $isNullVar ? ${ctx.defaultValue(dataType)} : ($value); + """ + } else { + s"$javaType $valueVar = $value;" + }).trim + ExprCode(code, isNullVar, valueVar) + } + + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. + * This produces an [[UnsafeRow]] for each row in each batch. + */ + // TODO: return ColumnarBatch.Rows instead + override protected def doProduce(ctx: CodegenContext): String = { + val input = ctx.freshName("input") + // PhysicalRDD always just has one input + ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") + + // metrics + val numOutputRows = metricTerm(ctx, "numOutputRows") + val scanTimeMetric = metricTerm(ctx, "scanTime") + val scanTimeTotalNs = ctx.freshName("scanTime") + ctx.addMutableState("long", scanTimeTotalNs, s"$scanTimeTotalNs = 0;") + val incReadBatches = if (!enableScanStatistics) "" else { + val readPartitions = ctx.addReferenceObj("readPartitions", inMemoryTableScan.readPartitions) + val readBatches = ctx.addReferenceObj("readBatches", inMemoryTableScan.readBatches) + ctx.addMutableState("int", "initializeInMemoryTableScanStatistics", + s""" + |$readPartitions.setValue(0); + |$readBatches.setValue(0); + |if ($input.hasNext()) { $readPartitions.add(1); } + """.stripMargin) + s"$readBatches.add(1);" + } + + val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" + val batch = ctx.freshName("batch") + ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") + + val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" + val idx = ctx.freshName("batchIdx") + ctx.addMutableState("int", idx, s"$idx = 0;") + val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) + val columnAssigns = colVars.zipWithIndex.map { case (name, i) => + ctx.addMutableState(columnVectorClz, name, s"$name = null;") + val index = if (columnIndexes == null) i else columnIndexes(i) + s"$name = $batch.column($index);" + } + + val nextBatch = ctx.freshName("nextBatch") + ctx.addNewFunction(nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | long getBatchStart = System.nanoTime(); + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $incReadBatches + | $numOutputRows.add($batch.numRows()); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | } + | $scanTimeTotalNs += System.nanoTime() - getBatchStart; + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) + } + s""" + |if ($batch == null) { + | $nextBatch(); + |} + |while ($batch != null) { + | int numRows = $batch.numRows(); + | while ($idx < numRows) { + | int $rowidx = $idx++; + | ${consume(ctx, columnsBatchInput).trim} + | if (shouldStop()) return; + | } + | $batch = null; + | $nextBatch(); + |} + |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); + |$scanTimeTotalNs = 0; + """.stripMargin + } + +} +>>>>>>> support UDT and column pruning diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index 470307bd940a..a5ba8a548b3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -62,6 +62,13 @@ private[columnar] sealed trait ColumnStats extends Serializable { count += 1 } + def gatherNullStats(): Unit = { + nullCount += 1 + // 1 bytes for null position + sizeInBytes += 1 + count += 1 + } + /** * Column statistics represented as a single row, currently including closed lower bound, closed * upper bound and null count. @@ -93,6 +100,15 @@ private[columnar] class BooleanColumnStats extends ColumnStats { } } + def gatherValueStats(value: Boolean): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += BOOLEAN.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -111,6 +127,15 @@ private[columnar] class ByteColumnStats extends ColumnStats { } } + def gatherValueStats(value: Byte): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += BYTE.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -129,6 +154,15 @@ private[columnar] class ShortColumnStats extends ColumnStats { } } + def gatherValueStats(value: Short): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += SHORT.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -147,6 +181,15 @@ private[columnar] class IntColumnStats extends ColumnStats { } } + def gatherValueStats(value: Int): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += INT.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -165,6 +208,15 @@ private[columnar] class LongColumnStats extends ColumnStats { } } + def gatherValueStats(value: Long): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += LONG.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -183,6 +235,15 @@ private[columnar] class FloatColumnStats extends ColumnStats { } } + def gatherValueStats(value: Float): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += FLOAT.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -201,6 +262,15 @@ private[columnar] class DoubleColumnStats extends ColumnStats { } } + def gatherValueStats(value: Double): Unit = { + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += DOUBLE.defaultSize + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -219,6 +289,15 @@ private[columnar] class StringColumnStats extends ColumnStats { } } + def gatherValueStats(value: UTF8String, size: Int): Unit = { + if (upper == null || value.compareTo(upper) > 0) upper = value.clone() + if (lower == null || value.compareTo(lower) < 0) lower = value.clone() + sizeInBytes += size + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -231,6 +310,13 @@ private[columnar] class BinaryColumnStats extends ColumnStats { } } + def gatherValueStats(value: Array[Byte], size: Int): Unit = { + sizeInBytes += size + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](null, null, nullCount, count, sizeInBytes)) } @@ -252,6 +338,16 @@ private[columnar] class DecimalColumnStats(precision: Int, scale: Int) extends C } } + def gatherValueStats(value: Decimal): Unit = { + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + // TODO: this is not right for DecimalType with precision > 18 + sizeInBytes += 8 + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) } @@ -266,6 +362,13 @@ private[columnar] class ObjectColumnStats(dataType: DataType) extends ColumnStat } } + def gatherValueStats(value: Object, size: Int): Unit = { + sizeInBytes += size + count += 1 + } + + def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](null, null, nullCount, count, sizeInBytes)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 999f1802a4b3..3ad011a46969 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -38,7 +38,7 @@ class GenerateColumnarBatch( schema: StructType, batchSize: Int, storageLevel: StorageLevel) - extends CodeGenerator[Iterator[InternalRow], Iterator[ColumnarBatch]] { + extends CodeGenerator[Iterator[InternalRow], Iterator[CachedColumnarBatch]] { protected def canonicalize(in: Iterator[InternalRow]): Iterator[InternalRow] = in @@ -48,7 +48,7 @@ class GenerateColumnarBatch( in } - protected def create(rowIterator: Iterator[InternalRow]): Iterator[ColumnarBatch] = { + protected def create(rowIterator: Iterator[InternalRow]): Iterator[CachedColumnarBatch] = { import scala.collection.JavaConverters._ val ctx = newCodeGenContext() val batchVar = ctx.freshName("columnarBatch") @@ -56,19 +56,53 @@ class GenerateColumnarBatch( val numBytesVar = ctx.freshName("bytesInBatch") val rowIterVar = ctx.addReferenceObj( "rowIterator", rowIterator.asJava, classOf[java.util.Iterator[_]].getName) - val schemaVar = ctx.addReferenceObj("schema", schema, classOf[StructType].getName) + val schemas = StructType( + schema.fields.map(s => StructField(s.name, + s.dataType match { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }, s.nullable)) + ) + val schemaVar = ctx.addReferenceObj("schema", schemas, classOf[StructType].getName) val maxNumBytes = ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE // Code to populate column vectors with the values of the input rows - val colVars = schema.fields.indices.map(i => ctx.freshName("colInstance" + i)) + val colVars = schemas.fields.indices.map(i => ctx.freshName("colInstance" + i)) val columnInstanceCode = colVars.zipWithIndex.map { case (name, i) => s"ColumnVector $name = $batchVar.column($i);" }.mkString("\n") - val populateColumnVectorsCode = (schema.fields zip colVars).zipWithIndex.map { - case ((field, colVar), i) => - GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, - colVar, "row", rowNumVar, i, numBytesVar) + val colStatVars = schemas.fields.indices.map(i => ctx.freshName("colStat" + i)) + val colStatCode = (schemas.fields zip colStatVars).zipWithIndex.map { + case ((field, varName), i) => + val (columnStatsCls, arg) = field.dataType match { + case BooleanType => (classOf[BooleanColumnStats].getName, "()") + case ByteType => (classOf[ByteColumnStats].getName, "()") + case ShortType => (classOf[ShortColumnStats].getName, "()") + case IntegerType | DateType => (classOf[IntColumnStats].getName, "()") + case LongType | TimestampType => (classOf[LongColumnStats].getName, "()") + case FloatType => (classOf[FloatColumnStats].getName, "()") + case DoubleType => (classOf[DoubleColumnStats].getName, "()") + case StringType => (classOf[StringColumnStats].getName, "()") + case BinaryType => (classOf[BinaryColumnStats].getName, "()") + case dt: DecimalType => + (classOf[DecimalColumnStats].getName, s"(${dt.precision}, ${dt.scale})") + case dt => (classOf[ObjectColumnStats].getName, s"(${dt})") + } + s"$columnStatsCls $varName = new $columnStatsCls$arg;" + }.mkString("") + val collectedStatistics = colStatVars.map(name => + s"$name.collectedStats()[0], $name.collectedStats()[1], " + + s"$name.collectedStats()[2], $name.collectedStats()[3], $name.collectedStats()[4]" + ).mkString("new Object[] { ", ", ", "}") + + val populateColumnVectorsCode = (schemas.fields, colVars, colStatVars).zipped + .toSeq.zipWithIndex.map { + case ((field, colVar, colStatVar), i) => + GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, + colVar, "row", rowNumVar, colStatVar, i, numBytesVar) }.mkString("") + + val code = s""" import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; @@ -92,10 +126,11 @@ class GenerateColumnarBatch( } @Override - public ColumnarBatch next() { + public ${classOf[CachedColumnarBatch].getName} next() { ColumnarBatch $batchVar = ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP_UNSAFE, $batchSize); $columnInstanceCode + $colStatCode int $rowNumVar = 0; long $numBytesVar = 0; while ($rowIterVar.hasNext() && $rowNumVar < $batchSize && $numBytesVar < $maxNumBytes) { @@ -104,14 +139,17 @@ class GenerateColumnarBatch( $rowNumVar += 1; } $batchVar.setNumRows($rowNumVar); - return $batchVar; + + // return $batchVar; + return ${classOf[CachedColumnarBatch].getName}.apply($batchVar, + new GenericInternalRow($collectedStatistics)); } } """ val formattedCode = CodeFormatter.stripOverlappingComments( new CodeAndComment(code, ctx.getPlaceHolderToComments())) CodeGenerator.compile(formattedCode).generate(ctx.references.toArray) - .asInstanceOf[Iterator[ColumnarBatch]] + .asInstanceOf[Iterator[CachedColumnarBatch]] } } @@ -134,21 +172,29 @@ private[columnar] object GenerateColumnarBatch { ) def putColumnCode(ctx: CodegenContext, dt: DataType, nullable: Boolean, colVar: String, - rowVar: String, rowNumVar: String, colNum: Int, numBytesVar: String) : String = { + rowVar: String, rowNumVar: String, colStatVar: String, colNum: Int, numBytesVar: String) + : String = { val body = dt match { case t if ctx.isPrimitiveType(dt) => val typeName = GenerateColumnarBatch.typeToName(dt) val put = "put" + typeName.capitalize val get = "get" + typeName.capitalize s""" - |$colVar.$put($rowNumVar, $rowVar.$get($colNum)); + |$typeName val = $rowVar.$get($colNum); + |$colVar.$put($rowNumVar, val); |$numBytesVar += ${dt.defaultSize}; + |$colStatVar.gatherValueStats(val); """.stripMargin case StringType | BinaryType => val typeName = GenerateColumnarBatch.typeToName(dt) val put = "put" + typeName.capitalize val get = "get" + typeName.capitalize - s"""$numBytesVar += $colVar.$put($rowNumVar, $rowVar.$get($colNum));""" + s""" + |$typeName val = $rowVar.$get($colNum); + |int size = $colVar.$put($rowNumVar, val); + |$numBytesVar += size; + |$colStatVar.gatherValueStats(val, size); + """.stripMargin case NullType => return s""" |if ($rowVar.isNullAt($colNum)) { @@ -181,12 +227,17 @@ private[columnar] object GenerateColumnarBatch { s""" |if ($rowVar.isNullAt($colNum)) { | $colVar.putNull($rowNumVar); + | $colStatVar.gatherNullStats(); |} else { | $body |} """.stripMargin } else { - body + s""" + |{ + | $body + |} + """.stripMargin } } } 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 efe1872f207e..911bc10ff367 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 @@ -55,7 +55,7 @@ private[columnar] case class CachedBatchBytes( /** * A cached batch of rows stored as a [[ColumnarBatch]]. */ -private[columnar] case class CachedColumnarBatch(columnarBatch: ColumnarBatch) +private[columnar] case class CachedColumnarBatch(columnarBatch: ColumnarBatch, stats: InternalRow) extends CachedBatch @@ -192,8 +192,10 @@ case class InMemoryRelation( private def buildColumnarBatches(): RDD[CachedColumnarBatch] = { val schema = StructType.fromAttributes(child.output) child.execute().mapPartitionsInternal { rows => - new GenerateColumnarBatch(schema, batchSize, storageLevel) - .generate(rows).map { columnarBatch => CachedColumnarBatch(columnarBatch) } + new GenerateColumnarBatch(schema, batchSize, storageLevel).generate(rows).map { + cachedColumnarBatch => batchStats.add(cachedColumnarBatch.stats) + cachedColumnarBatch + } }.persist(storageLevel) } 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 1854acd83c48..7303795d9454 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 @@ -36,16 +36,42 @@ case class InMemoryTableScanExec( override val columnIndexes = attributes.map(a => relation.output.indexOf(a)).toArray + override val inMemoryTableScan = this + override val supportCodegen: Boolean = relation.useColumnarBatches override def inputRDDs(): Seq[RDD[InternalRow]] = { if (relation.useColumnarBatches) { + val schema = relation.partitionStatistics.schema + val schemaIndex = schema.zipWithIndex + val buffers = relation.cachedColumnBuffers.asInstanceOf[RDD[CachedColumnarBatch]] + val prunedBuffers = if (inMemoryPartitionPruningEnabled) { + buffers.mapPartitionsInternal { cachedColumnarBatchIterator => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), schema) + + // Do partition batch pruning if enabled + cachedColumnarBatchIterator.filter { cachedColumnarBatch => + if (!partitionFilter(cachedColumnarBatch.stats)) { + def statsString: String = schemaIndex.map { + case (a, i) => + val value = cachedColumnarBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + logInfo(s"Skipping partition based on stats $statsString") + false + } else { + true + } + } + } + } else { + buffers + } + // HACK ALERT: This is actually an RDD[CachedColumnarBatch]. // We're taking advantage of Scala's type erasure here to pass these batches along. - Seq(relation.cachedColumnBuffers - .asInstanceOf[RDD[CachedColumnarBatch]] - .map(_.columnarBatch) - .asInstanceOf[RDD[InternalRow]]) + Seq(prunedBuffers.map(_.columnarBatch).asInstanceOf[RDD[InternalRow]]) } else { Seq() } From b5b9e753e2aad99b3e46d4620faa5dc4aecd8b30 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Sep 2016 00:43:44 +0900 Subject: [PATCH 24/40] fix build error --- .../apache/spark/sql/execution/benchmark/CacheBenchmark.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala index b904d09a4cc0..e728d2855fbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala @@ -121,7 +121,6 @@ class CacheBenchmark extends BenchmarkBase { // All of these are codegen = T hashmap = T sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.VECTORIZED_AGG_MAP_MAX_COLUMNS.key, "1024") // Benchmark cases: // (1) No caching From 9548c8affd9302288f824fc2b46951840b1eebcd Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Sep 2016 02:47:07 +0900 Subject: [PATCH 25/40] fix test failures --- .../sql/execution/columnar/ColumnStats.scala | 23 +++++++---- .../columnar/GenerateColumnarBatch.scala | 38 ++++++++++++++----- 2 files changed, 44 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index a5ba8a548b3b..dd46e151d587 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -292,7 +292,7 @@ private[columnar] class StringColumnStats extends ColumnStats { def gatherValueStats(value: UTF8String, size: Int): Unit = { if (upper == null || value.compareTo(upper) > 0) upper = value.clone() if (lower == null || value.compareTo(lower) < 0) lower = value.clone() - sizeInBytes += size + sizeInBytes += (size + 4) count += 1 } @@ -311,7 +311,7 @@ private[columnar] class BinaryColumnStats extends ColumnStats { } def gatherValueStats(value: Array[Byte], size: Int): Unit = { - sizeInBytes += size + sizeInBytes += (size + 4) count += 1 } @@ -338,11 +338,10 @@ private[columnar] class DecimalColumnStats(precision: Int, scale: Int) extends C } } - def gatherValueStats(value: Decimal): Unit = { + def gatherValueStats(value: Decimal, size: Int): Unit = { if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - // TODO: this is not right for DecimalType with precision > 18 - sizeInBytes += 8 + sizeInBytes += size count += 1 } @@ -362,6 +361,15 @@ private[columnar] class ObjectColumnStats(dataType: DataType) extends ColumnStat } } + override def collectedStatistics: GenericInternalRow = + new GenericInternalRow(Array[Any](null, null, nullCount, count, sizeInBytes)) +} + +private[columnar] class OtherColumnStats() extends ColumnStats { + override def gatherStats(row: InternalRow, ordinal: Int): Unit = { + throw new UnsupportedOperationException() + } + def gatherValueStats(value: Object, size: Int): Unit = { sizeInBytes += size count += 1 @@ -369,6 +377,7 @@ private[columnar] class ObjectColumnStats(dataType: DataType) extends ColumnStat def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) - override def collectedStatistics: GenericInternalRow = - new GenericInternalRow(Array[Any](null, null, nullCount, count, sizeInBytes)) + override def collectedStatistics: GenericInternalRow = { + throw new UnsupportedOperationException() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 3ad011a46969..89ea64ac6478 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -86,10 +86,10 @@ class GenerateColumnarBatch( case BinaryType => (classOf[BinaryColumnStats].getName, "()") case dt: DecimalType => (classOf[DecimalColumnStats].getName, s"(${dt.precision}, ${dt.scale})") - case dt => (classOf[ObjectColumnStats].getName, s"(${dt})") + case dt => (classOf[OtherColumnStats].getName, "()") } s"$columnStatsCls $varName = new $columnStatsCls$arg;" - }.mkString("") + }.mkString("\n") val collectedStatistics = colStatVars.map(name => s"$name.collectedStats()[0], $name.collectedStats()[1], " + s"$name.collectedStats()[2], $name.collectedStats()[3], $name.collectedStats()[4]" @@ -140,7 +140,6 @@ class GenerateColumnarBatch( } $batchVar.setNumRows($rowNumVar); - // return $batchVar; return ${classOf[CachedColumnarBatch].getName}.apply($batchVar, new GenericInternalRow($collectedStatistics)); } @@ -187,10 +186,14 @@ private[columnar] object GenerateColumnarBatch { """.stripMargin case StringType | BinaryType => val typeName = GenerateColumnarBatch.typeToName(dt) + val typeDeclName = dt match { + case StringType => "UTF8String" + case BinaryType => "byte[]" + } val put = "put" + typeName.capitalize val get = "get" + typeName.capitalize s""" - |$typeName val = $rowVar.$get($colNum); + |$typeDeclName val = $rowVar.$get($colNum); |int size = $colVar.$put($rowNumVar, val); |$numBytesVar += size; |$colStatVar.gatherValueStats(val, size); @@ -203,22 +206,37 @@ private[columnar] object GenerateColumnarBatch { | $colVar.putNotNull($rowNumVar); |} |$numBytesVar += 1; + |$colStatVar.gatherValueStats(null, 1); """.stripMargin case dt: DecimalType => val precision = dt.precision val scale = dt.scale s""" - $numBytesVar += $colVar.putDecimal($rowNumVar, - $rowVar.getDecimal($colNum, $precision, $scale), $precision); + |Decimal val = $rowVar.getDecimal($colNum, $precision, $scale); + |int size = $colVar.putDecimal($rowNumVar, val, $precision); + |$numBytesVar += size; + |$colStatVar.gatherValueStats(val, size); """.stripMargin case array: ArrayType => - s"""$numBytesVar += $colVar.putArray($rowNumVar, $rowVar.getArray($colNum));""" + s""" + |ArrayData val = $rowVar.getArray($colNum); + |int size = $colVar.putArray($rowNumVar, val); + |$numBytesVar += size; + |$colStatVar.gatherValueStats(val, size); + """.stripMargin case t: MapType => - s"""$numBytesVar += $colVar.putMap($rowNumVar, $rowVar.getMap($colNum));""" + s""" + |MapData val = $rowVar.getMap($colNum); + |int size = $colVar.putMap($rowNumVar, val); + |$numBytesVar += size; + |$colStatVar.gatherValueStats(val, size); + """.stripMargin case struct: StructType => s""" - $numBytesVar += $colVar.putStruct($rowNumVar, - $rowVar.getStruct($colNum, ${struct.length})); + |InternalRow val = $rowVar.getStruct($colNum, ${struct.length}); + |int size = $colVar.putStruct($rowNumVar,val); + |$numBytesVar += size; + |$colStatVar.gatherValueStats(val, size); """.stripMargin case _ => throw new UnsupportedOperationException("Unsupported data type " + dt.simpleString); From 3efea4498d884a8a30287ab711194e5c042dd782 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sun, 25 Sep 2016 23:42:04 +0900 Subject: [PATCH 26/40] fix test failures (union33, load_dyn_part13, multiMapJoin1) --- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 7303795d9454..31e2da40dfce 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 @@ -34,7 +34,8 @@ case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode with ColumnarBatchScan { - override val columnIndexes = attributes.map(a => relation.output.indexOf(a)).toArray + override val columnIndexes = + attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray override val inMemoryTableScan = this From 3c4959c47634d4f2490faa90cba959b521b992b6 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 27 Sep 2016 21:52:13 +0900 Subject: [PATCH 27/40] support compression of CachedColumnarBatch using CompressionCodec --- .../expressions/codegen/CodeGenerator.scala | 3 +- .../vectorized/OnHeapUnsafeColumnVector.java | 103 +++++++++++- .../sql/execution/ColumnarBatchScan.scala | 14 +- .../sql/execution/columnar/ColumnStats.scala | 28 ++-- .../columnar/GenerateColumnarBatch.scala | 151 ++++++++++++------ .../execution/columnar/InMemoryRelation.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 8 + .../columnar/InMemoryColumnarQuerySuite.scala | 18 ++- 8 files changed, 255 insertions(+), 74 deletions(-) mode change 100755 => 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java 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 f8f868b59b96..ad27ee748378 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 @@ -632,7 +632,8 @@ class CodegenContext { * @param row the variable name of row that is used by expressions * @param expressions the codes to evaluate expressions. */ - def splitExpressions(row: String, expressions: Seq[String]): String = { + def splitExpressions(row: String, expressions: Seq[String], + arguments: Seq[(String, String)] = null): String = { if (row == null || currentVars != null) { // Cannot split these expressions because they are not created from a row object. return expressions.mkString("\n") diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java old mode 100755 new mode 100644 index f49f490395d4..577ef2a4a55f --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java @@ -16,12 +16,14 @@ */ package org.apache.spark.sql.execution.vectorized; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Arrays; +import java.io.*; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.io.CompressionCodec; +import org.apache.spark.io.CompressionCodec$; import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; @@ -36,14 +38,19 @@ public final class OnHeapUnsafeColumnVector extends ColumnVector implements Seri // This is faster than a boolean array and we optimize this over memory footprint. private byte[] nulls; + private byte[] compressedNulls; // Array for all types private byte[] data; + private byte[] compressedData; // Only set if type is Array. private int[] arrayLengths; private int[] arrayOffsets; + private boolean compressed; + private transient CompressionCodec codec = null; + OnHeapUnsafeColumnVector() { } protected OnHeapUnsafeColumnVector(int capacity, DataType type) { @@ -65,18 +72,98 @@ public long nullsNativeAddress() { public void close() { } + public void compress(SparkConf conf) { + if (compressed) return; + if (codec == null) { + String codecName = conf.get(SQLConf.CACHE_COMPRESSION_CODEC()); + codec = CompressionCodec$.MODULE$.createCodec(conf, codecName); + } + ByteArrayOutputStream bos; + OutputStream out; + + if (data != null) { + bos = new ByteArrayOutputStream(); + out = codec.compressedOutputStream(bos); + try { + try { + out.write(data); + } finally { + out.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + if (bos.size() < data.length) { + compressedData = bos.toByteArray(); + data = null; + } + } + + if (nulls != null) { + bos = new ByteArrayOutputStream(); + out = codec.compressedOutputStream(bos); + try { + try { + out.write(nulls); + } finally { + out.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + if (bos.size() < nulls.length) { + compressedNulls = bos.toByteArray(); + nulls = null; + } + } + compressed = (compressedData != null) || (compressedNulls != null); + } + + public void decompress(SparkConf conf) throws IOException { + if (!compressed) return; + if (codec == null) { + String codecName = conf.get(SQLConf.CACHE_COMPRESSION_CODEC()); + codec = CompressionCodec$.MODULE$.createCodec(conf, codecName); + } + ByteArrayInputStream bis; + InputStream in; + + if (compressedData != null) { + bis = new ByteArrayInputStream(compressedData); + in = codec.compressedInputStream(bis); + try { + data = IOUtils.toByteArray(in); + } finally { + in.close(); + } + compressedData = null; + } + + if (compressedNulls != null) { + bis = new ByteArrayInputStream(compressedNulls); + in = codec.compressedInputStream(bis); + try { + nulls = IOUtils.toByteArray(in); + } finally { + in.close(); + } + compressedNulls = null; + } + compressed = false; + } + // // APIs dealing with nulls // @Override public void putNotNull(int rowId) { - nulls[rowId] = (byte)0; + Platform.putByte(nulls, Platform.BYTE_ARRAY_OFFSET + rowId, (byte)0); } @Override public void putNull(int rowId) { - nulls[rowId] = (byte)1; + Platform.putByte(nulls, Platform.BYTE_ARRAY_OFFSET + rowId, (byte)1); ++numNulls; anyNullsSet = true; } @@ -93,7 +180,7 @@ public void putNotNulls(int rowId, int count) { @Override public boolean isNullAt(int rowId) { - return nulls[rowId] == 1; + return Platform.getByte(nulls, Platform.BYTE_ARRAY_OFFSET + rowId) == 1; } // @@ -129,7 +216,7 @@ public void putByte(int rowId, byte value) { @Override public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { - data[i + rowId] = value; + Platform.putByte(data, Platform.BYTE_ARRAY_OFFSET + rowId + i, value); } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 47d91799c64f..088681628cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -170,11 +170,12 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { package org.apache.spark.sql.execution +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector, OnHeapUnsafeColumnVector} import org.apache.spark.sql.types.DataType @@ -252,6 +253,13 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { val batch = ctx.freshName("batch") ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") + val generateDecompress = if (inMemoryTableScan != null) true else false + val confVar = if (!generateDecompress) null else { + val conf = inMemoryTableScan.sparkContext.conf + ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) + } + val onHeapUnsafeColumnVectorCls = classOf[OnHeapUnsafeColumnVector].getName + val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" val idx = ctx.freshName("batchIdx") ctx.addMutableState("int", idx, s"$idx = 0;") @@ -259,7 +267,9 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { val columnAssigns = colVars.zipWithIndex.map { case (name, i) => ctx.addMutableState(columnVectorClz, name, s"$name = null;") val index = if (columnIndexes == null) i else columnIndexes(i) - s"$name = $batch.column($index);" + val decompress = if (!generateDecompress) "" + else s" (($onHeapUnsafeColumnVectorCls)$name).decompress($confVar);" + s"$name = $batch.column($index);$decompress" } val nextBatch = ctx.freshName("nextBatch") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index dd46e151d587..3197d0c0358e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -74,6 +74,8 @@ private[columnar] sealed trait ColumnStats extends Serializable { * upper bound and null count. */ def collectedStatistics: GenericInternalRow + + def collectedStats: Array[Any] } /** @@ -82,6 +84,8 @@ private[columnar] sealed trait ColumnStats extends Serializable { private[columnar] class NoopColumnStats extends ColumnStats { override def gatherStats(row: InternalRow, ordinal: Int): Unit = super.gatherStats(row, ordinal) + override def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, 0L) + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](null, null, nullCount, count, 0L)) } @@ -107,7 +111,7 @@ private[columnar] class BooleanColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -134,7 +138,7 @@ private[columnar] class ByteColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -161,7 +165,7 @@ private[columnar] class ShortColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -188,7 +192,7 @@ private[columnar] class IntColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -215,7 +219,7 @@ private[columnar] class LongColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -242,7 +246,7 @@ private[columnar] class FloatColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -269,7 +273,7 @@ private[columnar] class DoubleColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -296,7 +300,7 @@ private[columnar] class StringColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -315,7 +319,7 @@ private[columnar] class BinaryColumnStats extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](null, null, nullCount, count, sizeInBytes)) @@ -345,7 +349,7 @@ private[columnar] class DecimalColumnStats(precision: Int, scale: Int) extends C count += 1 } - def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](lower, upper, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](lower, upper, nullCount, count, sizeInBytes)) @@ -361,6 +365,8 @@ private[columnar] class ObjectColumnStats(dataType: DataType) extends ColumnStat } } + override def collectedStats: Array[Any] = null + override def collectedStatistics: GenericInternalRow = new GenericInternalRow(Array[Any](null, null, nullCount, count, sizeInBytes)) } @@ -375,7 +381,7 @@ private[columnar] class OtherColumnStats() extends ColumnStats { count += 1 } - def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) + override def collectedStats: Array[Any] = Array[Any](null, null, nullCount, count, sizeInBytes) override def collectedStatistics: GenericInternalRow = { throw new UnsupportedOperationException() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 89ea64ac6478..d16d22d4bc76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.execution.columnar +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.execution.vectorized.ColumnarBatch +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, OnHeapUnsafeColumnVector} import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.storage.StorageLevel._ /** @@ -37,7 +39,8 @@ abstract class ColumnarBatchIterator extends Iterator[ColumnarBatch] class GenerateColumnarBatch( schema: StructType, batchSize: Int, - storageLevel: StorageLevel) + storageLevel: StorageLevel, + conf: SparkConf) extends CodeGenerator[Iterator[InternalRow], Iterator[CachedColumnarBatch]] { protected def canonicalize(in: Iterator[InternalRow]): Iterator[InternalRow] = in @@ -51,9 +54,12 @@ class GenerateColumnarBatch( protected def create(rowIterator: Iterator[InternalRow]): Iterator[CachedColumnarBatch] = { import scala.collection.JavaConverters._ val ctx = newCodeGenContext() + val columnStatsCls = classOf[ColumnStats].getName + val rowVar = ctx.freshName("row") val batchVar = ctx.freshName("columnarBatch") val rowNumVar = ctx.freshName("rowNum") val numBytesVar = ctx.freshName("bytesInBatch") + ctx.addMutableState("long", numBytesVar, s"$numBytesVar = 0;") val rowIterVar = ctx.addReferenceObj( "rowIterator", rowIterator.asJava, classOf[java.util.Iterator[_]].getName) val schemas = StructType( @@ -65,49 +71,64 @@ class GenerateColumnarBatch( ) val schemaVar = ctx.addReferenceObj("schema", schemas, classOf[StructType].getName) val maxNumBytes = ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE - // Code to populate column vectors with the values of the input rows - val colVars = schemas.fields.indices.map(i => ctx.freshName("colInstance" + i)) - val columnInstanceCode = colVars.zipWithIndex.map { case (name, i) => - s"ColumnVector $name = $batchVar.column($i);" - }.mkString("\n") + val numColumns = schema.fields.length - val colStatVars = schemas.fields.indices.map(i => ctx.freshName("colStat" + i)) - val colStatCode = (schemas.fields zip colStatVars).zipWithIndex.map { - case ((field, varName), i) => - val (columnStatsCls, arg) = field.dataType match { - case BooleanType => (classOf[BooleanColumnStats].getName, "()") - case ByteType => (classOf[ByteColumnStats].getName, "()") - case ShortType => (classOf[ShortColumnStats].getName, "()") - case IntegerType | DateType => (classOf[IntColumnStats].getName, "()") - case LongType | TimestampType => (classOf[LongColumnStats].getName, "()") - case FloatType => (classOf[FloatColumnStats].getName, "()") - case DoubleType => (classOf[DoubleColumnStats].getName, "()") - case StringType => (classOf[StringColumnStats].getName, "()") - case BinaryType => (classOf[BinaryColumnStats].getName, "()") - case dt: DecimalType => - (classOf[DecimalColumnStats].getName, s"(${dt.precision}, ${dt.scale})") - case dt => (classOf[OtherColumnStats].getName, "()") - } - s"$columnStatsCls $varName = new $columnStatsCls$arg;" - }.mkString("\n") - val collectedStatistics = colStatVars.map(name => - s"$name.collectedStats()[0], $name.collectedStats()[1], " + - s"$name.collectedStats()[2], $name.collectedStats()[3], $name.collectedStats()[4]" - ).mkString("new Object[] { ", ", ", "}") + val colStatVars = (0 to numColumns - 1).map(i => ctx.freshName("colStat" + i)) + val colStatCode = ctx.splitExpressions( + "row", + (schemas.fields zip colStatVars).zipWithIndex.map { + case ((field, varName), i) => + val (columnStatsCls, arg) = field.dataType match { + case BooleanType => (classOf[BooleanColumnStats].getName, "()") + case ByteType => (classOf[ByteColumnStats].getName, "()") + case ShortType => (classOf[ShortColumnStats].getName, "()") + case IntegerType | DateType => (classOf[IntColumnStats].getName, "()") + case LongType | TimestampType => (classOf[LongColumnStats].getName, "()") + case FloatType => (classOf[FloatColumnStats].getName, "()") + case DoubleType => (classOf[DoubleColumnStats].getName, "()") + case StringType => (classOf[StringColumnStats].getName, "()") + case BinaryType => (classOf[BinaryColumnStats].getName, "()") + case dt: DecimalType => + (classOf[DecimalColumnStats].getName, s"(${dt.precision}, ${dt.scale})") + case dt => (classOf[OtherColumnStats].getName, "()") + } + ctx.addMutableState(columnStatsCls, varName, "") + s"$varName = new $columnStatsCls$arg;\n" + } + ) + val assignCollectedStatistics = ctx.splitExpressions( + "row", + colStatVars.zipWithIndex.map { case (name, i) => + s"assignStats(array, $name, $i);\n" + }, + Seq(("Object[]", "array")) + ) + val numColStats = colStatVars.length * 5 - val populateColumnVectorsCode = (schemas.fields, colVars, colStatVars).zipped - .toSeq.zipWithIndex.map { - case ((field, colVar, colStatVar), i) => - GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, - colVar, "row", rowNumVar, colStatVar, i, numBytesVar) - }.mkString("") + val populateColumnVectorsCode = ctx.splitExpressions( + rowVar, + (schemas.fields zip colStatVars).zipWithIndex.map { + case ((field, colStatVar), i) => + GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, + batchVar, rowVar, rowNumVar, colStatVar, i, numBytesVar) + }, + Seq(("ColumnarBatch", batchVar), ("int", rowNumVar)) + ) + val confVar = ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) + val compress = + s""" + for (int i = 0; i < $numColumns; i++) { + ((OnHeapUnsafeColumnVector)$batchVar.column(i)).compress($confVar); + } + """ val code = s""" import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.vectorized.ColumnarBatch; import org.apache.spark.sql.execution.vectorized.ColumnVector; + import org.apache.spark.sql.execution.vectorized.OnHeapUnsafeColumnVector; public GeneratedColumnarBatchIterator generate(Object[] references) { return new GeneratedColumnarBatchIterator(references); @@ -120,6 +141,30 @@ class GenerateColumnarBatch( ${ctx.initMutableStates()} } + ${ctx.declareAddedFunctions()} + + private void allocateColumnStats() { + InternalRow row = null; + $colStatCode + } + + private void assignStats(Object[] array, $columnStatsCls stat, int i) { + Object[] stats = stat.collectedStats(); + int idx = i * 5; + array[idx] = stats[0]; + array[idx+1] = stats[1]; + array[idx+2] = stats[2]; + array[idx+3] = stats[3]; + array[idx+4] = stats[4]; + } + + private InternalRow allocateStats() { + InternalRow row = null; + Object[] array = new Object[$numColStats]; + $assignCollectedStatistics + return new GenericInternalRow(array); + } + @Override public boolean hasNext() { return $rowIterVar.hasNext(); @@ -128,20 +173,18 @@ class GenerateColumnarBatch( @Override public ${classOf[CachedColumnarBatch].getName} next() { ColumnarBatch $batchVar = - ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP_UNSAFE, $batchSize); - $columnInstanceCode - $colStatCode + ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP_UNSAFE, $batchSize); + allocateColumnStats(); int $rowNumVar = 0; - long $numBytesVar = 0; + $numBytesVar = 0; while ($rowIterVar.hasNext() && $rowNumVar < $batchSize && $numBytesVar < $maxNumBytes) { - InternalRow row = (InternalRow) $rowIterVar.next(); + InternalRow $rowVar = (InternalRow) $rowIterVar.next(); $populateColumnVectorsCode $rowNumVar += 1; } $batchVar.setNumRows($rowNumVar); - - return ${classOf[CachedColumnarBatch].getName}.apply($batchVar, - new GenericInternalRow($collectedStatistics)); + $compress + return ${classOf[CachedColumnarBatch].getName}.apply($batchVar, allocateStats()); } } """ @@ -154,7 +197,22 @@ class GenerateColumnarBatch( } -private[columnar] object GenerateColumnarBatch { +private[sql] object GenerateColumnarBatch { + + def compressStorageLevel(storageLevel: StorageLevel): StorageLevel = { + storageLevel match { + case MEMORY_ONLY => MEMORY_ONLY_SER + case MEMORY_ONLY_2 => MEMORY_ONLY_SER_2 + case MEMORY_AND_DISK => MEMORY_AND_DISK_SER + case MEMORY_AND_DISK_2 => MEMORY_AND_DISK_SER_2 + case sl => sl + } + } + + def isCompress(storageLevel: StorageLevel) : Boolean = { + (storageLevel == MEMORY_ONLY_SER || storageLevel == MEMORY_ONLY_SER_2 || + storageLevel == MEMORY_AND_DISK_SER || storageLevel == MEMORY_AND_DISK_SER_2) + } private val typeToName = Map[AbstractDataType, String]( BooleanType -> "boolean", @@ -170,9 +228,10 @@ private[columnar] object GenerateColumnarBatch { BinaryType -> "Binary" ) - def putColumnCode(ctx: CodegenContext, dt: DataType, nullable: Boolean, colVar: String, + def putColumnCode(ctx: CodegenContext, dt: DataType, nullable: Boolean, batchVar: String, rowVar: String, rowNumVar: String, colStatVar: String, colNum: Int, numBytesVar: String) : String = { + val colVar = s"$batchVar.column($colNum)" val body = dt match { case t if ctx.isPrimitiveType(dt) => val typeName = GenerateColumnarBatch.typeToName(dt) 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 911bc10ff367..9fb1bfca9a01 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 @@ -191,8 +191,10 @@ case class InMemoryRelation( */ private def buildColumnarBatches(): RDD[CachedColumnarBatch] = { val schema = StructType.fromAttributes(child.output) + val newStorageLevel = GenerateColumnarBatch.compressStorageLevel(storageLevel) + val conf = child.sqlContext.sparkSession.sparkContext.conf child.execute().mapPartitionsInternal { rows => - new GenerateColumnarBatch(schema, batchSize, storageLevel).generate(rows).map { + new GenerateColumnarBatch(schema, batchSize, newStorageLevel, conf).generate(rows).map { cachedColumnarBatch => batchStats.add(cachedColumnarBatch.stats) cachedColumnarBatch } 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 21ab6fdd9ac5..f1a54d424ab6 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 @@ -97,6 +97,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val CACHE_COMPRESSION_CODEC = + SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.compression.codec") + .internal() + .doc("Sets the compression codec use when columnar caching is compressed.") + .stringConf + .transform(_.toLowerCase()) + .createWithDefault("lz4") + val PREFER_SORTMERGEJOIN = SQLConfigBuilder("spark.sql.join.preferSortMergeJoin") .internal() .doc("When true, prefer sort merge join over shuffle hash join.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 45bbb6cbe2e0..e3e05159e294 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, GenericInternalRow} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ @@ -116,7 +116,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { data.createOrReplaceTempView(s"testData$dataType") val useColumnBatches = true withSQLConf(SQLConf.CACHE_CODEGEN.key -> useColumnBatches.toString) { - Seq(MEMORY_ONLY).map { storageLevel => + Seq(MEMORY_ONLY, MEMORY_ONLY_SER).map { storageLevel => val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan val inMemoryRelation = InMemoryRelation(useCompression = false, 5, storageLevel, plan, None) @@ -424,8 +424,15 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val length2 = 8117 val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) + } - /* TODO: test GenerateColumnarBatch().generate() with many columns */ + test("GenerateColumnarBatch.generate() with many columns") { + val length1 = 9000 + val schema = StructType((1 to length1).map { case i => + StructField(s"col$i", IntegerType, true) + }) + val cachedBatch1 = new GenerateColumnarBatch(schema, 10000, MEMORY_ONLY, sparkConf). + generate(Iterator.single(new GenericInternalRow((1 to length1).toArray[Any]))) } test("SPARK-17549: cached table size should be correctly calculated") { @@ -441,6 +448,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(cached.batchStats.value === expectedAnswer.size * INT.defaultSize) } +<<<<<<< d3dd78fee6464d6c97485a00cda984d3ed67c07e test("access primitive-type columns in CachedBatch without whole stage codegen") { // whole stage codegen is not applied to a row with more than WHOLESTAGE_MAX_NUM_FIELDS fields withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "2") { @@ -500,12 +508,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } } - test("InMemoryRelation builds the correct buffers") { + test("InMemoryRelation builds the correct buffers with simple schemas") { testColumnBatches(useColumnBatches = true, useComplexSchema = false) testColumnBatches(useColumnBatches = false, useComplexSchema = false) } - test("InMemoryRelation falls back on non-codegen path with complex schemas") { + test("InMemoryRelation builds the correct buffers with complex schemas") { testColumnBatches(useColumnBatches = true, useComplexSchema = true) testColumnBatches(useColumnBatches = false, useComplexSchema = true) } From ca7abf55f55fad79d3c8f76299e050ef095cd92e Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Sep 2016 11:28:32 +0900 Subject: [PATCH 28/40] make generated code smaller --- .../columnar/GenerateColumnarBatch.scala | 149 ++++++++---------- 1 file changed, 63 insertions(+), 86 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index d16d22d4bc76..3bf8ab1ec577 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -93,36 +93,21 @@ class GenerateColumnarBatch( case dt => (classOf[OtherColumnStats].getName, "()") } ctx.addMutableState(columnStatsCls, varName, "") - s"$varName = new $columnStatsCls$arg;\n" + s"$varName = new $columnStatsCls$arg; statsArray[$i] = $varName;\n" } ) - val assignCollectedStatistics = ctx.splitExpressions( - "row", - colStatVars.zipWithIndex.map { case (name, i) => - s"assignStats(array, $name, $i);\n" - }, - Seq(("Object[]", "array")) - ) - val numColStats = colStatVars.length * 5 val populateColumnVectorsCode = ctx.splitExpressions( rowVar, (schemas.fields zip colStatVars).zipWithIndex.map { case ((field, colStatVar), i) => GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, - batchVar, rowVar, rowNumVar, colStatVar, i, numBytesVar) + batchVar, rowVar, rowNumVar, colStatVar, i, numBytesVar).trim }, Seq(("ColumnarBatch", batchVar), ("int", rowNumVar)) ) val confVar = ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) - val compress = - s""" - for (int i = 0; i < $numColumns; i++) { - ((OnHeapUnsafeColumnVector)$batchVar.column(i)).compress($confVar); - } - """ - val code = s""" import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; @@ -143,26 +128,10 @@ class GenerateColumnarBatch( ${ctx.declareAddedFunctions()} + $columnStatsCls[] statsArray = new $columnStatsCls[$numColumns]; private void allocateColumnStats() { InternalRow row = null; - $colStatCode - } - - private void assignStats(Object[] array, $columnStatsCls stat, int i) { - Object[] stats = stat.collectedStats(); - int idx = i * 5; - array[idx] = stats[0]; - array[idx+1] = stats[1]; - array[idx+2] = stats[2]; - array[idx+3] = stats[3]; - array[idx+4] = stats[4]; - } - - private InternalRow allocateStats() { - InternalRow row = null; - Object[] array = new Object[$numColStats]; - $assignCollectedStatistics - return new GenericInternalRow(array); + ${colStatCode.trim} } @Override @@ -183,8 +152,11 @@ class GenerateColumnarBatch( $rowNumVar += 1; } $batchVar.setNumRows($rowNumVar); - $compress - return ${classOf[CachedColumnarBatch].getName}.apply($batchVar, allocateStats()); + for (int i = 0; i < $numColumns; i++) { + ((OnHeapUnsafeColumnVector)$batchVar.column(i)).compress($confVar); + } + return ${classOf[CachedColumnarBatch].getName}.apply( + $batchVar, ${classOf[GenerateColumnarBatch].getName}.generateStats(statsArray)); } } """ @@ -238,11 +210,11 @@ private[sql] object GenerateColumnarBatch { val put = "put" + typeName.capitalize val get = "get" + typeName.capitalize s""" - |$typeName val = $rowVar.$get($colNum); - |$colVar.$put($rowNumVar, val); - |$numBytesVar += ${dt.defaultSize}; - |$colStatVar.gatherValueStats(val); - """.stripMargin + $typeName val = $rowVar.$get($colNum); + $colVar.$put($rowNumVar, val); + $numBytesVar += ${dt.defaultSize}; + $colStatVar.gatherValueStats(val); + """ case StringType | BinaryType => val typeName = GenerateColumnarBatch.typeToName(dt) val typeDeclName = dt match { @@ -252,69 +224,74 @@ private[sql] object GenerateColumnarBatch { val put = "put" + typeName.capitalize val get = "get" + typeName.capitalize s""" - |$typeDeclName val = $rowVar.$get($colNum); - |int size = $colVar.$put($rowNumVar, val); - |$numBytesVar += size; - |$colStatVar.gatherValueStats(val, size); - """.stripMargin + $typeDeclName val = $rowVar.$get($colNum); + int size = $colVar.$put($rowNumVar, val); + $numBytesVar += size; + $colStatVar.gatherValueStats(val, size); + """ case NullType => return s""" - |if ($rowVar.isNullAt($colNum)) { - | $colVar.putNull($rowNumVar); - |} else { - | $colVar.putNotNull($rowNumVar); - |} - |$numBytesVar += 1; - |$colStatVar.gatherValueStats(null, 1); - """.stripMargin + if ($rowVar.isNullAt($colNum)) { + $colVar.putNull($rowNumVar); + } else { + $colVar.putNotNull($rowNumVar); + } + $numBytesVar += 1; + $colStatVar.gatherValueStats(null, 1); + """ case dt: DecimalType => val precision = dt.precision val scale = dt.scale s""" - |Decimal val = $rowVar.getDecimal($colNum, $precision, $scale); - |int size = $colVar.putDecimal($rowNumVar, val, $precision); - |$numBytesVar += size; - |$colStatVar.gatherValueStats(val, size); - """.stripMargin + Decimal val = $rowVar.getDecimal($colNum, $precision, $scale); + int size = $colVar.putDecimal($rowNumVar, val, $precision); + $numBytesVar += size; + $colStatVar.gatherValueStats(val, size); + """ case array: ArrayType => s""" - |ArrayData val = $rowVar.getArray($colNum); - |int size = $colVar.putArray($rowNumVar, val); - |$numBytesVar += size; - |$colStatVar.gatherValueStats(val, size); - """.stripMargin + ArrayData val = $rowVar.getArray($colNum); + int size = $colVar.putArray($rowNumVar, val); + $numBytesVar += size; + $colStatVar.gatherValueStats(val, size); + """ case t: MapType => s""" - |MapData val = $rowVar.getMap($colNum); - |int size = $colVar.putMap($rowNumVar, val); - |$numBytesVar += size; - |$colStatVar.gatherValueStats(val, size); - """.stripMargin + MapData val = $rowVar.getMap($colNum); + int size = $colVar.putMap($rowNumVar, val); + $numBytesVar += size; + $colStatVar.gatherValueStats(val, size); + """ case struct: StructType => s""" - |InternalRow val = $rowVar.getStruct($colNum, ${struct.length}); - |int size = $colVar.putStruct($rowNumVar,val); - |$numBytesVar += size; - |$colStatVar.gatherValueStats(val, size); - """.stripMargin + InternalRow val = $rowVar.getStruct($colNum, ${struct.length}); + int size = $colVar.putStruct($rowNumVar,val); + $numBytesVar += size; + $colStatVar.gatherValueStats(val, size); + """ case _ => throw new UnsupportedOperationException("Unsupported data type " + dt.simpleString); } if (nullable) { s""" - |if ($rowVar.isNullAt($colNum)) { - | $colVar.putNull($rowNumVar); - | $colStatVar.gatherNullStats(); - |} else { - | $body - |} - """.stripMargin + if ($rowVar.isNullAt($colNum)) { + $colVar.putNull($rowNumVar); + $colStatVar.gatherNullStats(); + } else { + ${body.trim} + } + """ } else { s""" - |{ - | $body - |} - """.stripMargin + { + ${body.trim} + } + """ } } + + def generateStats(columnStats: Array[ColumnStats]): InternalRow = { + val array = columnStats.map(_.collectedStats).flatten + InternalRow.fromSeq(array) + } } From 9aa5c9bc84991856def8b60d26f9b59e3f6280b9 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Sep 2016 11:30:20 +0900 Subject: [PATCH 29/40] Generate Java code of an iterator for CachedColumnarBatch When whole stage codegen is not enabled, an iterator is still used whole CacheColumnarBatch is used --- .../vectorized/OnHeapUnsafeColumnVector.java | 24 ++- .../columnar/GenerateColumnAccessor.scala | 191 +++++++++++++++++- .../execution/columnar/InMemoryRelation.scala | 13 +- .../columnar/InMemoryTableScanExec.scala | 8 +- .../columnar/InMemoryColumnarQuerySuite.scala | 153 ++++++++------ 5 files changed, 310 insertions(+), 79 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java index 577ef2a4a55f..5c9f40e0cfd5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapUnsafeColumnVector.java @@ -119,7 +119,7 @@ public void compress(SparkConf conf) { compressed = (compressedData != null) || (compressedNulls != null); } - public void decompress(SparkConf conf) throws IOException { + public void decompress(SparkConf conf) { if (!compressed) return; if (codec == null) { String codecName = conf.get(SQLConf.CACHE_COMPRESSION_CODEC()); @@ -132,10 +132,14 @@ public void decompress(SparkConf conf) throws IOException { bis = new ByteArrayInputStream(compressedData); in = codec.compressedInputStream(bis); try { - data = IOUtils.toByteArray(in); - } finally { - in.close(); - } + try { + data = IOUtils.toByteArray(in); + } finally { + in.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } compressedData = null; } @@ -143,9 +147,13 @@ public void decompress(SparkConf conf) throws IOException { bis = new ByteArrayInputStream(compressedNulls); in = codec.compressedInputStream(bis); try { - nulls = IOUtils.toByteArray(in); - } finally { - in.close(); + try { + nulls = IOUtils.toByteArray(in); + } finally { + in.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); } compressedNulls = null; } 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 72bd20418830..be744c818270 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,11 +17,16 @@ package org.apache.spark.sql.execution.columnar +import org.apache.spark.SparkConf 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._ +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.execution.vectorized.ColumnarBatch import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.types.UTF8String /** * An Iterator to walk through the InternalRows from a CachedBatch @@ -57,17 +62,56 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends BaseGenericInternalR override protected def genericGet(ordinal: Int): Any = throw new UnsupportedOperationException override def numFields: Int = throw new UnsupportedOperationException override def copy(): InternalRow = throw new UnsupportedOperationException + + def setDecimal(i: Int, v: Decimal, precision: Int, scale: Int): Unit = + writer.write(i, v, precision, scale) + def setUTF8String(i: Int, s: UTF8String): Unit = writer.write(i, s) + def setBinary(i: Int, b: Array[Byte]): Unit = writer.write(i, b) + def setArray(i: Int, a: ArrayData): Unit = { + val u = a.asInstanceOf[UnsafeArrayData] + val base = u.getBaseObject.asInstanceOf[Array[Byte]] + val offset = u.getBaseOffset - Platform.BYTE_ARRAY_OFFSET + if (offset > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot write this array as it's too big.") + } + val size = u.getSizeInBytes + writer.write(i, base, offset.toInt, size) + } + def setMap(i: Int, m: MapData): Unit = { + val u = m.asInstanceOf[UnsafeMapData] + val base = u.getBaseObject.asInstanceOf[Array[Byte]] + val offset = u.getBaseOffset - Platform.BYTE_ARRAY_OFFSET + if (offset > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot write this array as it's too big.") + } + val size = u.getSizeInBytes + writer.write(i, base, offset.toInt, size) + } + def setStruct(i: Int, r: MutableRow): Unit = { + val u = r.asInstanceOf[UnsafeRow] + val base = u.getBaseObject.asInstanceOf[Array[Byte]] + val offset = u.getBaseOffset - Platform.BYTE_ARRAY_OFFSET + if (offset > Integer.MAX_VALUE) { + throw new UnsupportedOperationException("Cannot write this array as it's too big.") + } + val size = u.getSizeInBytes + writer.write(i, base, offset.toInt, size) + } } /** * Generates bytecode for a [[ColumnarIterator]] for columnar cache. */ -object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarIterator] with Logging { +class GenerateColumnAccessor(conf: SparkConf) + extends CodeGenerator[Seq[DataType], ColumnarIterator] with Logging { protected def canonicalize(in: Seq[DataType]): Seq[DataType] = in protected def bind(in: Seq[DataType], inputSchema: Seq[Attribute]): Seq[DataType] = in protected def create(columnTypes: Seq[DataType]): ColumnarIterator = { + if (conf != null) { + return createItrForCacheColumnarBatch(conf, columnTypes) + } val ctx = newCodeGenContext() val numFields = columnTypes.size val (initializeAccessors, extractors) = columnTypes.zipWithIndex.map { case (dt, index) => @@ -229,8 +273,149 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera val code = CodeFormatter.stripOverlappingComments( new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) - logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") + logDebug(s"Generated ColumnarIteratorForCachedBatchBytes:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] } + + protected def createItrForCacheColumnarBatch(conf: SparkConf, columnTypes: Seq[DataType]) + : ColumnarIterator = { + val ctx = newCodeGenContext() + val numFields = columnTypes.size + val confVar = ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) + + val setters = ctx.splitExpressions( + "row", + columnTypes.zipWithIndex.map { case (dt, index) => + val setter = dt match { + case NullType => + s"if (colInstances[$index].isNullAt(rowIdx)) { mutableRow.setNullAt($index); }\n" + case BooleanType => s"setBoolean($index, colInstances[$index].getBoolean(rowIdx))" + case ByteType => s"setByte($index, colInstances[$index].getByte(rowIdx))" + case ShortType => s"setShort($index, colInstances[$index].getShort(rowIdx))" + case IntegerType | DateType => s"setInt($index, colInstances[$index].getInt(rowIdx))" + case LongType | TimestampType => s"setLong($index, colInstances[$index].getLong(rowIdx))" + case FloatType => s"setFloat($index, colInstances[$index].getFloat(rowIdx))" + case DoubleType => s"setDouble($index, colInstances[$index].getDouble(rowIdx))" + case dt: DecimalType if dt.precision <= Decimal.MAX_INT_DIGITS => + s"setLong($index, (long)colInstances[$index].getInt(rowIdx))" + case dt: DecimalType if dt.precision <= Decimal.MAX_LONG_DIGITS => + s"setLong($index, colInstances[$index].getLong(rowIdx))" + case dt: DecimalType => + val p = dt.precision + val s = dt.scale + s"setDecimal($index, colInstances[$index].getDecimal(rowIdx, $p, $s), $p, $s)" + case StringType => s"setUTF8String($index, colInstances[$index].getUTF8String(rowIdx))" + case BinaryType => s"setBinary($index, colInstances[$index].getBinary(rowIdx))" + case array: ArrayType => s"setArray($index, colInstances[$index].getArray(rowIdx))" + case t: MapType => s"setMap($index, colInstances[$index].getMap(rowIdx))" + case struct: StructType => + val s = struct.fields.length + s"setStruct($index, colInstances[$index].getStruct(rowIdx, $s))" + } + + dt match { + case NullType => setter + case dt: DecimalType if dt.precision > Decimal.MAX_LONG_DIGITS => + s""" + if (colInstances[$index].isNullAt(rowIdx)) { + mutableRow.setDecimal($index, null, ${dt.precision}, ${dt.scale}); + } else { + mutableRow.$setter; + } + """ + case _ => + s""" + if (colInstances[$index].isNullAt(rowIdx)) { + mutableRow.setNullAt($index); + } else { + mutableRow.$setter; + } + """ + } + } + ) + + val codeBody = s""" + import scala.collection.Iterator; + 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.MutableUnsafeRow; + import org.apache.spark.sql.execution.vectorized.ColumnVector; + import org.apache.spark.sql.execution.vectorized.OnHeapUnsafeColumnVector; + + public SpecificColumnarIterator generate(Object[] references) { + return new SpecificColumnarIterator(references); + } + + class SpecificColumnarIterator extends ${classOf[ColumnarIterator].getName} { + private ColumnVector[] colInstances; + private UnsafeRow unsafeRow = new UnsafeRow($numFields); + private BufferHolder bufferHolder = new BufferHolder(unsafeRow); + private UnsafeRowWriter rowWriter = new UnsafeRowWriter(bufferHolder, $numFields); + private MutableUnsafeRow mutableRow = null; + + private int rowIdx = 0; + private int numRowsInBatch = 0; + + private scala.collection.Iterator input = null; + private DataType[] columnTypes = null; + private int[] columnIndexes = null; + + ${ctx.declareMutableStates()} + + public SpecificColumnarIterator(Object[] references) { + ${ctx.initMutableStates()} + this.mutableRow = new MutableUnsafeRow(rowWriter); + } + + public void initialize(Iterator input, DataType[] columnTypes, int[] columnIndexes) { + this.input = input; + this.columnTypes = columnTypes; + this.columnIndexes = columnIndexes; + } + + ${ctx.declareAddedFunctions()} + + public boolean hasNext() { + if (rowIdx < numRowsInBatch) { + return true; + } + if (!input.hasNext()) { + return false; + } + + ${classOf[CachedColumnarBatch].getName} cachedBatch = + (${classOf[CachedColumnarBatch].getName}) input.next(); + ${classOf[ColumnarBatch].getName} batch = cachedBatch.columnarBatch(); + rowIdx = 0; + numRowsInBatch = cachedBatch.getNumRows(); + colInstances = new ColumnVector[columnIndexes.length]; + for (int i = 0; i < columnIndexes.length; i ++) { + colInstances[i] = batch.column(columnIndexes[i]); + ((OnHeapUnsafeColumnVector)colInstances[i]).decompress($confVar); + } + + return hasNext(); + } + + public InternalRow next() { + bufferHolder.reset(); + rowWriter.zeroOutNullBytes(); + InternalRow row = null; + ${setters} + unsafeRow.setTotalSize(bufferHolder.totalSize()); + rowIdx += 1; + return unsafeRow; + } + }""" + + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) + logDebug(s"Generated ColumnarIteratorForCachedColumnarBatch:\n${CodeFormatter.format(code)}") + + CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[ColumnarIterator] + } + } 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 9fb1bfca9a01..b92925359de6 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 @@ -37,7 +37,10 @@ import org.apache.spark.util.LongAccumulator /** * An abstract representation of a cached batch of rows. */ -private[columnar] trait CachedBatch +private[columnar] trait CachedBatch { + val stats: InternalRow + def getNumRows(): Int +} /** @@ -49,14 +52,18 @@ private[columnar] trait CachedBatch */ private[columnar] case class CachedBatchBytes( numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) - extends CachedBatch + extends CachedBatch { + def getNumRows(): Int = numRows +} /** * A cached batch of rows stored as a [[ColumnarBatch]]. */ private[columnar] case class CachedColumnarBatch(columnarBatch: ColumnarBatch, stats: InternalRow) - extends CachedBatch + extends CachedBatch { + def getNumRows(): Int = columnarBatch.numRows() +} object InMemoryRelation { 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 31e2da40dfce..d0c4633fc976 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 @@ -163,7 +163,6 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning protected override def doExecute(): RDD[InternalRow] = { - assert(!relation.useColumnarBatches) val numOutputRows = longMetric("numOutputRows") if (enableAccumulators) { @@ -177,7 +176,8 @@ case class InMemoryTableScanExec( val schemaIndex = schema.zipWithIndex val relOutput: AttributeSeq = relation.output assert(relation.cachedColumnBuffers != null) - val buffers = relation.cachedColumnBuffers.asInstanceOf[RDD[CachedBatchBytes]] + val buffers = relation.cachedColumnBuffers + val conf = if (relation.useColumnarBatches) sqlContext.sparkContext.conf else null buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => val partitionFilter = newPredicate( @@ -216,7 +216,7 @@ case class InMemoryTableScanExec( if (enableAccumulators) { readBatches.add(1) } - numOutputRows += batch.numRows + numOutputRows += batch.getNumRows() batch } @@ -224,7 +224,7 @@ case class InMemoryTableScanExec( case udt: UserDefinedType[_] => udt.sqlType case other => other }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + val columnarIterator = new GenerateColumnAccessor(conf).generate(columnTypes) columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) if (enableAccumulators && columnarIterator.hasNext) { readPartitions.add(1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index e3e05159e294..a6aca00d7f03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -136,7 +136,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { // all primitives Seq(true, false).map { nullability => val dataTypes = Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, - FloatType, DoubleType) + FloatType, DoubleType, DateType, TimestampType, DecimalType(25, 5), DecimalType(6, 5)) val schema = StructType(dataTypes.zipWithIndex.map { case (dataType, index) => StructField(s"col$index", dataType, nullability) }) @@ -147,74 +147,39 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { if (nullability && i % 3 == 0) null else i.toInt, if (nullability && i % 3 == 0) null else i.toLong, if (nullability && i % 3 == 0) null else (i + 0.25).toFloat, - if (nullability && i % 3 == 0) null else (i + 0.75).toDouble - ))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "primitives") - } - - val schemaNull = StructType(Seq(StructField(s"col", NullType, true))) - val rddNull = spark.sparkContext.parallelize((1 to 10).map(i => Row(null))) - cachePrimitiveTest(spark.createDataFrame(rddNull, schemaNull), "Null") - - Seq(true, false).map { nullability => - val schema = StructType(Seq(StructField(s"col0", DateType, nullability), - StructField(s"col1", TimestampType, nullability))) - val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else (i + 0.75).toDouble, if (nullability && i % 3 == 0) null else new Date(i), - if (nullability && i % 3 == 0) null else new Timestamp(i * 1000000L) - ))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "DateTimestamp") - } - - Seq(true, false).map { nullability => - val schema = StructType(Seq(StructField(s"col", StringType, nullability))) - val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( - if (nullability && i % 3 == 0) null else s"str${i}: test cache."))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "String") - } - - Seq(true, false).map { nullability => - val schema = StructType(Seq(StructField(s"col0", DecimalType(25, 5), nullability), - StructField(s"col1", DecimalType(6, 5), nullability))) - val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else new Timestamp(i * 1000000L), if (nullability && i % 3 == 0) null else BigDecimal(Long.MaxValue.toString + ".12345"), if (nullability && i % 3 == 0) null else new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456") ))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Decimal") + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "primitivesDateTimeStamp") } - Seq(true, false).map { nullability => - val schema = StructType(Seq(StructField(s"col", ArrayType(IntegerType), nullability))) - val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( - if (nullability && i % 3 == 0) null else (i * 100 to i * 100 + i).toArray))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Array") - } - - Seq(true, false).map { nullability => - val schema = StructType( - Seq(StructField(s"col", ArrayType(ArrayType(IntegerType)), nullability))) - val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( - if (nullability && i % 3 == 0) null - else Array(Array(i, i + 1), Array(i * 100 + 1, i * 100, i * 100 + 2))))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "ArrayArray") - } - - Seq(true, false).map { nullability => - val schema = StructType( - Seq(StructField(s"col", MapType(StringType, IntegerType), nullability))) - val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( - if (nullability && i % 3 == 0) null else (i to i + i).map(j => s"key$j" -> j).toMap))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Map") - } + val schemaNull = StructType(Seq(StructField("col", NullType, true))) + val rddNull = spark.sparkContext.parallelize((1 to 10).map(i => Row(null))) + cachePrimitiveTest(spark.createDataFrame(rddNull, schemaNull), "Null") Seq(true, false).map { nullability => val struct = StructType(StructField("f1", FloatType, false) :: StructField("f2", ArrayType(BooleanType), true) :: Nil) - val schema = StructType(Seq(StructField(s"col", struct, nullability))) + val schema = StructType(Seq( + StructField("col0", StringType, nullability), + StructField("col1", ArrayType(IntegerType), nullability), + StructField("col2", ArrayType(ArrayType(IntegerType)), nullability), + StructField("col3", MapType(StringType, IntegerType), nullability), + StructField("col4", struct, nullability) + )) val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( - if (nullability && i % 3 == 0) null else Row((i + 0.25).toFloat, Seq(true, false, null))))) - cachePrimitiveTest(spark.createDataFrame(rdd, schema), "Struct") + if (nullability && i % 3 == 0) null else s"str${i}: test cache.", + if (nullability && i % 3 == 0) null else (i * 100 to i * 100 + i).toArray, + if (nullability && i % 3 == 0) null + else Array(Array(i, i + 1), Array(i * 100 + 1, i * 100, i * 100 + 2)), + if (nullability && i % 3 == 0) null else (i to i + i).map(j => s"key$j" -> j).toMap, + if (nullability && i % 3 == 0) null else Row((i + 0.25).toFloat, Seq(true, false, null)) + ))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "StringArrayMapStruct") } } @@ -418,21 +383,88 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-14138: Generated SpecificColumnarIterator can exceed JVM size limit for cached DF") { val length1 = 3999 val columnTypes1 = List.fill(length1)(IntegerType) - val columnarIterator1 = GenerateColumnAccessor.generate(columnTypes1) + val columnarIterator1 = new GenerateColumnAccessor(null).generate(columnTypes1) // SPARK-16664: the limit of janino is 8117 val length2 = 8117 val columnTypes2 = List.fill(length2)(IntegerType) - val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) + val columnarIterator2 = new GenerateColumnAccessor(null).generate(columnTypes2) } - test("GenerateColumnarBatch.generate() with many columns") { + test("ColumnarBatch with many columns") { val length1 = 9000 val schema = StructType((1 to length1).map { case i => StructField(s"col$i", IntegerType, true) }) val cachedBatch1 = new GenerateColumnarBatch(schema, 10000, MEMORY_ONLY, sparkConf). generate(Iterator.single(new GenericInternalRow((1 to length1).toArray[Any]))) + + val length2 = 9000 + val columnTypes2 = List.fill(length2)(IntegerType) + val columnarIterator2 = new GenerateColumnAccessor(sparkConf).generate(columnTypes2) + } + + test("access only some column of the all of columns") { + val df = spark.range(1, 100).map(i => (i, (i + 1).toFloat)).toDF("i", "f").cache + df.count + assert(df.filter("f <= 10.0").count == 9) + } + + test("access columns in CachedColumnarBatch without whole stage codegen") { + // whole stage codegen is not applied to a row with more than WHOLESTAGE_MAX_NUM_FIELDS fields + val dummySeq = Seq.range(0, 20) + val dummySchemas = dummySeq.map(i => StructField(s"d$i" + i, IntegerType, true)) + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "20") { + val data = Seq(null, true, 1.toByte, 3.toShort, 7, 15.toLong, + 31.25.toFloat, 63.75, new Date(127), new Timestamp(255000000L), null) + val dataTypes = Seq(NullType, BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DateType, TimestampType, IntegerType) + val schemas = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data ++ dummySeq))) + val df = spark.createDataFrame(rdd, StructType(schemas ++ dummySchemas)) + val row = df.persist.take(1).apply(0) + checkAnswer(df, row) + } + + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "20") { + val data = Seq(BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal("1234567890.12345"), + new java.math.BigDecimal("1.23456"), + "test123" + ) + val schemas = Seq( + StructField("col0", DecimalType(25, 5), true), + StructField("col1", DecimalType(15, 5), true), + StructField("col2", DecimalType(6, 5), true), + StructField("col3", StringType, true) + ) + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data ++ dummySeq))) + val df = spark.createDataFrame(rdd, StructType(schemas ++ dummySchemas)) + val row = df.persist.take(1).apply(0) + checkAnswer(df, row) + } + + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "20") { + val data = Seq((1 to 10).toArray, + Array(Array(10, 11), Array(100, 111, 123)), + Map("key1" -> 111, "key2" -> 222), + Row(1.25.toFloat, Seq(true, false, null)) + ) + val struct = StructType(StructField("f1", FloatType, false) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val schemas = Seq( + StructField("col0", ArrayType(IntegerType), true), + StructField("col1", ArrayType(ArrayType(IntegerType)), true), + StructField("col2", MapType(StringType, IntegerType), true), + StructField("col3", struct, true) + ) + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data ++ dummySeq))) + val df = spark.createDataFrame(rdd, StructType(schemas ++ dummySchemas)) + val row = df.persist.take(1).apply(0) + checkAnswer(df, row) + } } test("SPARK-17549: cached table size should be correctly calculated") { @@ -545,5 +577,4 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } } } - } From 1154aa527b7e731aeafe8281947d6231efa5bfa5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 29 Sep 2016 14:12:50 +0900 Subject: [PATCH 30/40] replace test() with ignore() --- .../apache/spark/sql/execution/benchmark/CacheBenchmark.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala index e728d2855fbd..16ad7f57e6ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala @@ -25,7 +25,7 @@ import org.apache.spark.util.Benchmark class CacheBenchmark extends BenchmarkBase { - test("cache with randomized keys - end-to-end") { + ignore("cache with randomized keys - end-to-end") { benchmarkRandomizedKeys(size = 20 << 18, readPathOnly = false) /* @@ -41,7 +41,7 @@ class CacheBenchmark extends BenchmarkBase { */ } - test("cache with randomized keys - read path only") { + ignore("cache with randomized keys - read path only") { benchmarkRandomizedKeys(size = 20 << 21, readPathOnly = true) /* From c00e42e6fe8093c16f986ed02892cc13fd99a56e Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 30 Sep 2016 02:32:07 +0900 Subject: [PATCH 31/40] support uncompressed ColumnVector --- .../columnar/GenerateColumnAccessor.scala | 4 ++-- .../columnar/GenerateColumnarBatch.scala | 23 ++++++++++++------- .../execution/columnar/InMemoryRelation.scala | 2 +- 3 files changed, 18 insertions(+), 11 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 be744c818270..5b0b0bf80167 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 @@ -203,6 +203,7 @@ class GenerateColumnAccessor(conf: SparkConf) 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.CachedBatchBytes; import org.apache.spark.sql.execution.columnar.MutableUnsafeRow; public SpecificColumnarIterator generate(Object[] references) { @@ -249,8 +250,7 @@ class GenerateColumnAccessor(conf: SparkConf) return false; } - ${classOf[CachedBatchBytes].getName} batch = - (${classOf[CachedBatchBytes].getName}) input.next(); + CachedBatchBytes batch = (CachedBatchBytes)input.next(); currentRow = 0; numRowsInBatch = batch.numRows(); for (int i = 0; i < columnIndexes.length; i ++) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index 3bf8ab1ec577..a728215642f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -102,15 +102,23 @@ class GenerateColumnarBatch( (schemas.fields zip colStatVars).zipWithIndex.map { case ((field, colStatVar), i) => GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, - batchVar, rowVar, rowNumVar, colStatVar, i, numBytesVar).trim + batchVar, rowVar, rowNumVar, colStatVar, i, numBytesVar).trim + "\n" }, Seq(("ColumnarBatch", batchVar), ("int", rowNumVar)) ) val confVar = ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) + val compress = if (!GenerateColumnarBatch.isCompress(storageLevel)) "" else s""" + for (int i = 0; i < $numColumns; i++) { + ((OnHeapUnsafeColumnVector)$batchVar.column(i)).compress($confVar); + } + """ + val code = s""" import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; + import org.apache.spark.sql.execution.columnar.CachedColumnarBatch; + import org.apache.spark.sql.execution.columnar.GenerateColumnarBatch; import org.apache.spark.sql.execution.vectorized.ColumnarBatch; import org.apache.spark.sql.execution.vectorized.ColumnVector; import org.apache.spark.sql.execution.vectorized.OnHeapUnsafeColumnVector; @@ -140,7 +148,7 @@ class GenerateColumnarBatch( } @Override - public ${classOf[CachedColumnarBatch].getName} next() { + public CachedColumnarBatch next() { ColumnarBatch $batchVar = ColumnarBatch.allocate($schemaVar, MemoryMode.ON_HEAP_UNSAFE, $batchSize); allocateColumnStats(); @@ -152,11 +160,9 @@ class GenerateColumnarBatch( $rowNumVar += 1; } $batchVar.setNumRows($rowNumVar); - for (int i = 0; i < $numColumns; i++) { - ((OnHeapUnsafeColumnVector)$batchVar.column(i)).compress($confVar); - } - return ${classOf[CachedColumnarBatch].getName}.apply( - $batchVar, ${classOf[GenerateColumnarBatch].getName}.generateStats(statsArray)); + ${compress.trim} + return CachedColumnarBatch.apply( + $batchVar, GenerateColumnarBatch.generateStats(statsArray)); } } """ @@ -171,7 +177,8 @@ class GenerateColumnarBatch( private[sql] object GenerateColumnarBatch { - def compressStorageLevel(storageLevel: StorageLevel): StorageLevel = { + def compressStorageLevel(storageLevel: StorageLevel, useCompression: Boolean): StorageLevel = { + if (!useCompression) return storageLevel storageLevel match { case MEMORY_ONLY => MEMORY_ONLY_SER case MEMORY_ONLY_2 => MEMORY_ONLY_SER_2 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 b92925359de6..170d05aa69be 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 @@ -198,7 +198,7 @@ case class InMemoryRelation( */ private def buildColumnarBatches(): RDD[CachedColumnarBatch] = { val schema = StructType.fromAttributes(child.output) - val newStorageLevel = GenerateColumnarBatch.compressStorageLevel(storageLevel) + val newStorageLevel = GenerateColumnarBatch.compressStorageLevel(storageLevel, useCompression) val conf = child.sqlContext.sparkSession.sparkContext.conf child.execute().mapPartitionsInternal { rows => new GenerateColumnarBatch(schema, batchSize, newStorageLevel, conf).generate(rows).map { From fdd2b1cccad88d1c4b4a37eaf3011a15e87f4c7e Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 30 Sep 2016 02:32:34 +0900 Subject: [PATCH 32/40] update benchmark programs --- .../org/apache/spark/util/Benchmark.scala | 31 ++++--- .../execution/benchmark/CacheBenchmark.scala | 93 +++++++++---------- 2 files changed, 65 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 7576faa99c96..7d900938204c 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -106,12 +106,7 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - try { - c.prepare() - measure(valuesPerIteration, c.numIters)(c.fn) - } finally { - c.cleanup() - } + measure(valuesPerIteration, c.numIters, c.prepare, c.cleanup)(c.fn) } println @@ -138,21 +133,33 @@ private[spark] class Benchmark( * Runs a single function `f` for iters, returning the average time the function took and * the rate of the function. */ - def measure(num: Long, overrideNumIters: Int)(f: Timer => Unit): Result = { + def measure(num: Long, overrideNumIters: Int, prepare: () => Unit, cleanup: () => Unit) + (f: Timer => Unit): Result = { System.gc() // ensures garbage from previous cases don't impact this one val warmupDeadline = warmupTime.fromNow while (!warmupDeadline.isOverdue) { - f(new Benchmark.Timer(-1)) + try { + prepare() + f(new Benchmark.Timer(-1)) + } finally { + cleanup() + } } val minIters = if (overrideNumIters != 0) overrideNumIters else minNumIters val minDuration = if (overrideNumIters != 0) 0 else minTime.toNanos val runTimes = ArrayBuffer[Long]() var i = 0 while (i < minIters || runTimes.sum < minDuration) { - val timer = new Benchmark.Timer(i) - f(timer) - val runTime = timer.totalTime() - runTimes += runTime + val runTime = try { + prepare() + val timer = new Benchmark.Timer(i) + f(timer) + val time = timer.totalTime() + runTimes += time + time + } finally { + cleanup() + } if (outputPerIteration) { // scalastyle:off diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala index 16ad7f57e6ed..0477bf08f28c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CacheBenchmark.scala @@ -25,52 +25,47 @@ import org.apache.spark.util.Benchmark class CacheBenchmark extends BenchmarkBase { - ignore("cache with randomized keys - end-to-end") { - benchmarkRandomizedKeys(size = 20 << 18, readPathOnly = false) + ignore("cache with randomized keys - both build and read paths") { + benchmarkRandomizedKeys(size = 16 * 1024 * 1024, readPathOnly = false) /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------- - cache = F 641 / 667 8.2 122.2 1.0X - cache = T columnar_batches = F compress = F 1696 / 1833 3.1 323.6 0.4X - cache = T columnar_batches = F compress = T 7517 / 7748 0.7 1433.8 0.1X - cache = T columnar_batches = T 1023 / 1102 5.1 195.0 0.6X + cache = T columnarBatch = F compress = T 7211 / 7366 2.3 429.8 1.0X + cache = T columnarBatch = F compress = F 2381 / 2460 7.0 141.9 3.0X + cache = F 137 / 140 122.7 8.1 52.7X + cache = T columnarBatch = T compress = T 2109 / 2252 8.0 125.7 3.4X + cache = T columnarBatch = T compress = F 1126 / 1184 14.9 67.1 6.4X */ } ignore("cache with randomized keys - read path only") { - benchmarkRandomizedKeys(size = 20 << 21, readPathOnly = true) + benchmarkRandomizedKeys(size = 64 * 1024 * 1024, readPathOnly = true) /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ----------------------------------------------------------------------------------------------- - cache = F 890 / 920 47.1 21.2 1.0X - cache = T columnar_batches = F compress = F 1950 / 1978 21.5 46.5 0.5X - cache = T columnar_batches = F compress = T 1893 / 1927 22.2 45.1 0.5X - cache = T columnar_batches = T 540 / 544 77.7 12.9 1.6X + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Cache random keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ---------------------------------------------------------------------------------------------- + cache = T columnarBatch = F compress = T 1615 / 1655 41.5 24.1 1.0X + cache = T columnarBatch = F compress = F 1603 / 1690 41.9 23.9 1.0X + cache = F 444 / 449 151.3 6.6 3.6X + cache = T columnarBatch = T compress = T 1404 / 1526 47.8 20.9 1.2X + cache = T columnarBatch = T compress = F 116 / 125 579.0 1.7 13.9X */ } /** - * Call collect on a [[DataFrame]] after deleting all existing temporary files. - * This also checks whether the collected result matches the expected answer. + * Call clean on a [[DataFrame]] after deleting all existing temporary files. */ - private def collect(df: DataFrame, expectedAnswer: Seq[Row]): Unit = { + private def clean(df: DataFrame): Unit = { df.sparkSession.sparkContext.parallelize(1 to 10, 10).foreach { _ => SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach { dir => dir.delete() } } - QueryTest.checkAnswer(df, expectedAnswer) match { - case Some(errMessage) => throw new RuntimeException(errMessage) - case None => // all good - } } /** @@ -88,34 +83,32 @@ class CacheBenchmark extends BenchmarkBase { .selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") val query = "select count(k), count(id) from test" - val expectedAnswer = sparkSession.sql(query).collect().toSeq /** * Add a benchmark case, optionally specifying whether to cache the dataset. */ def addBenchmark(name: String, cache: Boolean, params: Map[String, String] = Map()): Unit = { val ds = sparkSession.sql(query) + var dsResult: DataFrame = null val defaults = params.keys.flatMap { k => sparkSession.conf.getOption(k).map((k, _)) } def prepare(): Unit = { + clean(ds) params.foreach { case (k, v) => sparkSession.conf.set(k, v) } - if (cache) { sparkSession.catalog.cacheTable("test") } - if (readPathOnly) { - collect(ds, expectedAnswer) + if (cache && readPathOnly) { + sparkSession.sql("cache table test") } } def cleanup(): Unit = { + clean(dsResult) defaults.foreach { case (k, v) => sparkSession.conf.set(k, v) } sparkSession.catalog.clearCache() } benchmark.addCase(name, numIters, prepare, cleanup) { _ => - if (readPathOnly) { - collect(ds, expectedAnswer) - } else { - // also benchmark the time it takes to build the column buffers - val ds2 = sparkSession.sql(query) - collect(ds2, expectedAnswer) - collect(ds2, expectedAnswer) + if (cache && !readPathOnly) { + sparkSession.sql("cache table test") } + dsResult = sparkSession.sql(query) + dsResult.collect } } @@ -123,21 +116,27 @@ class CacheBenchmark extends BenchmarkBase { sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") // Benchmark cases: - // (1) No caching + // (1) Caching with compression // (2) Caching without compression - // (3) Caching with compression - // (4) Caching with column batches (without compression) - addBenchmark("cache = F", cache = false) - addBenchmark("cache = T columnar_batches = F compress = F", cache = true, Map( + // (3) No caching + // (4) Caching using column batch with compression + // (5) Caching using column batch without compression + addBenchmark("cache = T columnarBatch = F compress = T", cache = true, Map( SQLConf.CACHE_CODEGEN.key -> "false", - SQLConf.COMPRESS_CACHED.key -> "false" + SQLConf.COMPRESS_CACHED.key -> "true" )) - addBenchmark("cache = T columnar_batches = F compress = T", cache = true, Map( + addBenchmark("cache = T columnarBatch = F compress = F", cache = true, Map( SQLConf.CACHE_CODEGEN.key -> "false", + SQLConf.COMPRESS_CACHED.key -> "false" + )) + addBenchmark("cache = F", cache = false) + addBenchmark("cache = T columnarBatch = T compress = T", cache = true, Map( + SQLConf.CACHE_CODEGEN.key -> "true", SQLConf.COMPRESS_CACHED.key -> "true" )) - addBenchmark("cache = T columnar_batches = T", cache = true, Map( - SQLConf.CACHE_CODEGEN.key -> "true" + addBenchmark("cache = T columnarBatch = T compress = F", cache = true, Map( + SQLConf.CACHE_CODEGEN.key -> "true", + SQLConf.COMPRESS_CACHED.key -> "false" )) benchmark.run() } From 572abd94f00a2ac9108dc428a50acee9d5af521b Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 4 Oct 2016 20:33:14 +0900 Subject: [PATCH 33/40] rebase --- .../catalyst/expressions/codegen/CodeGenerator.scala | 3 +-- .../execution/columnar/GenerateColumnAccessor.scala | 6 +++--- .../execution/columnar/GenerateColumnarBatch.scala | 12 +++++++----- 3 files changed, 11 insertions(+), 10 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 ad27ee748378..f8f868b59b96 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 @@ -632,8 +632,7 @@ class CodegenContext { * @param row the variable name of row that is used by expressions * @param expressions the codes to evaluate expressions. */ - def splitExpressions(row: String, expressions: Seq[String], - arguments: Seq[(String, String)] = null): String = { + def splitExpressions(row: String, expressions: Seq[String]): String = { if (row == null || currentVars != null) { // Cannot split these expressions because they are not created from a row object. return expressions.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 5b0b0bf80167..575254ec58e5 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 @@ -285,7 +285,6 @@ class GenerateColumnAccessor(conf: SparkConf) val confVar = ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) val setters = ctx.splitExpressions( - "row", columnTypes.zipWithIndex.map { case (dt, index) => val setter = dt match { case NullType => @@ -333,7 +332,9 @@ class GenerateColumnAccessor(conf: SparkConf) } """ } - } + }, + "apply", + Seq.empty ) val codeBody = s""" @@ -403,7 +404,6 @@ class GenerateColumnAccessor(conf: SparkConf) public InternalRow next() { bufferHolder.reset(); rowWriter.zeroOutNullBytes(); - InternalRow row = null; ${setters} unsafeRow.setTotalSize(bufferHolder.totalSize()); rowIdx += 1; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala index a728215642f0..e6fad0a9d80b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnarBatch.scala @@ -75,7 +75,6 @@ class GenerateColumnarBatch( val colStatVars = (0 to numColumns - 1).map(i => ctx.freshName("colStat" + i)) val colStatCode = ctx.splitExpressions( - "row", (schemas.fields zip colStatVars).zipWithIndex.map { case ((field, varName), i) => val (columnStatsCls, arg) = field.dataType match { @@ -94,17 +93,19 @@ class GenerateColumnarBatch( } ctx.addMutableState(columnStatsCls, varName, "") s"$varName = new $columnStatsCls$arg; statsArray[$i] = $varName;\n" - } + }, + "apply", + Seq.empty ) val populateColumnVectorsCode = ctx.splitExpressions( - rowVar, (schemas.fields zip colStatVars).zipWithIndex.map { case ((field, colStatVar), i) => GenerateColumnarBatch.putColumnCode(ctx, field.dataType, field.nullable, batchVar, rowVar, rowNumVar, colStatVar, i, numBytesVar).trim + "\n" }, - Seq(("ColumnarBatch", batchVar), ("int", rowNumVar)) + "apply", + Seq(("InternalRow", rowVar), ("ColumnarBatch", batchVar), ("int", rowNumVar)) ) val confVar = ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) @@ -128,9 +129,11 @@ class GenerateColumnarBatch( } class GeneratedColumnarBatchIterator extends ${classOf[ColumnarBatchIterator].getName} { + private Object[] references; ${ctx.declareMutableStates()} public GeneratedColumnarBatchIterator(Object[] references) { + this.references = references; ${ctx.initMutableStates()} } @@ -138,7 +141,6 @@ class GenerateColumnarBatch( $columnStatsCls[] statsArray = new $columnStatsCls[$numColumns]; private void allocateColumnStats() { - InternalRow row = null; ${colStatCode.trim} } From 750dc729c72a9162f36a95c0178341b6afb5b591 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 5 Oct 2016 02:53:24 +0900 Subject: [PATCH 34/40] rebase --- .../sql/execution/columnar/InMemoryRelation.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 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 170d05aa69be..cf706dde1b83 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 @@ -202,8 +202,16 @@ case class InMemoryRelation( val conf = child.sqlContext.sparkSession.sparkContext.conf child.execute().mapPartitionsInternal { rows => new GenerateColumnarBatch(schema, batchSize, newStorageLevel, conf).generate(rows).map { - cachedColumnarBatch => batchStats.add(cachedColumnarBatch.stats) - cachedColumnarBatch + cachedColumnarBatch => { + var i = 0 + var totalSize = 0L + while (i < cachedColumnarBatch.columnarBatch.numCols()) { + totalSize += cachedColumnarBatch.stats.getLong(4 + i * 5) + i += 1 + } + batchStats.add(totalSize) + cachedColumnarBatch + } } }.persist(storageLevel) } From b573ec6f039d953d7ff84e37e1a60563b86d644d Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 8 Oct 2016 20:30:53 +0900 Subject: [PATCH 35/40] fix compilation errors --- .../apache/spark/sql/execution/vectorized/ColumnVector.java | 3 +-- .../spark/sql/execution/columnar/GenerateColumnAccessor.scala | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) 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 9c7d4c397dfd..9c0082e15635 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 @@ -26,7 +26,6 @@ import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.MutableRow; import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData; import org.apache.spark.sql.catalyst.expressions.UnsafeMapData; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; @@ -558,7 +557,7 @@ public ColumnarBatch.Row getStruct(int rowId) { * Returns a utility object to get structs. * provided to keep API compatibility with InternalRow for code generation */ - public MutableRow getStruct(int rowId, int size) { + public InternalRow getStruct(int rowId, int size) { if (!unsafeDirectCopy) { resultStruct.rowId = rowId; return resultStruct; 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 575254ec58e5..bbb99223df43 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 @@ -87,7 +87,7 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends BaseGenericInternalR val size = u.getSizeInBytes writer.write(i, base, offset.toInt, size) } - def setStruct(i: Int, r: MutableRow): Unit = { + def setStruct(i: Int, r: InternalRow): Unit = { val u = r.asInstanceOf[UnsafeRow] val base = u.getBaseObject.asInstanceOf[Array[Byte]] val offset = u.getBaseOffset - Platform.BYTE_ARRAY_OFFSET From 9066c3f9663f290185699edeab1c6f216061728a Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 1 Dec 2016 22:53:30 +0900 Subject: [PATCH 36/40] rebase --- .../sql/execution/columnar/InMemoryColumnarQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index a6aca00d7f03..3abf6e7a3f04 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -315,7 +315,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val allColumns = fields.map(_.name).mkString(",") val schema = StructType(fields) - // Create an RDD for the schema + // Create a RDD for the schema val rdd = sparkContext.parallelize((1 to 10000), 10).map { i => Row( From 65011cb730a055f4f2848bf5d8c02da710ecaed5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 1 Dec 2016 22:57:52 +0900 Subject: [PATCH 37/40] fix typo --- .../sql/execution/columnar/InMemoryColumnarQuerySuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 3abf6e7a3f04..12232bf22ac4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -480,7 +480,6 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(cached.batchStats.value === expectedAnswer.size * INT.defaultSize) } -<<<<<<< d3dd78fee6464d6c97485a00cda984d3ed67c07e test("access primitive-type columns in CachedBatch without whole stage codegen") { // whole stage codegen is not applied to a row with more than WHOLESTAGE_MAX_NUM_FIELDS fields withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "2") { From e4407c79c9138bdea5c50a31e81a041ad98b94d3 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 2 Dec 2016 00:59:29 +0900 Subject: [PATCH 38/40] merge --- .../columnar/InMemoryTableScanExec.scala | 2 +- .../columnar/InMemoryColumnarQuerySuite.scala | 50 +++++++------------ 2 files changed, 19 insertions(+), 33 deletions(-) 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 d0c4633fc976..440410683f5d 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 @@ -53,7 +53,7 @@ case class InMemoryTableScanExec( // Do partition batch pruning if enabled cachedColumnarBatchIterator.filter { cachedColumnarBatch => - if (!partitionFilter(cachedColumnarBatch.stats)) { + if (!partitionFilter.eval(cachedColumnarBatch.stats)) { def statsString: String = schemaIndex.map { case (a, i) => val value = cachedColumnarBatch.stats.get(i, a.dataType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 12232bf22ac4..fc375bc36f71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -34,18 +34,24 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { setupTestData() - private def cachePrimitiveTest(data: DataFrame, dataType: String) { + def cachePrimitiveTest(data: DataFrame, dataType: String) { data.createOrReplaceTempView(s"testData$dataType") - val storageLevel = MEMORY_ONLY - val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None) + val useColumnBatches = true + withSQLConf(SQLConf.CACHE_CODEGEN.key -> useColumnBatches.toString) { + Seq(MEMORY_ONLY, MEMORY_ONLY_SER).map { storageLevel => + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val inMemoryRelation = InMemoryRelation(useCompression = false, 5, storageLevel, plan, None) - assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == storageLevel) - inMemoryRelation.cachedColumnBuffers.collect().head match { - case _: CachedBatch => - case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") + assert(inMemoryRelation.useColumnarBatches == useColumnBatches) + assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == storageLevel) + inMemoryRelation.cachedColumnBuffers.collect().head match { + case _: CachedColumnarBatch => assert(useColumnBatches) + case _: CachedBatchBytes => assert(!useColumnBatches) + case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") + } + checkAnswer(inMemoryRelation, data.collect().toSeq) + } } - checkAnswer(inMemoryRelation, data.collect().toSeq) } private def testPrimitiveType(nullability: Boolean): Unit = { @@ -71,7 +77,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { cachePrimitiveTest(spark.createDataFrame(rdd, schema), "primitivesDateTimeStamp") } - private def tesNonPrimitiveType(nullability: Boolean): Unit = { + private def testNonPrimitiveType(nullability: Boolean): Unit = { val struct = StructType(StructField("f1", FloatType, false) :: StructField("f2", ArrayType(BooleanType), true) :: Nil) val schema = StructType(Seq( @@ -105,31 +111,11 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val rddNull = spark.sparkContext.parallelize((1 to 10).map(i => Row(null))) cachePrimitiveTest(spark.createDataFrame(rddNull, schemaNull), "Null") - tesNonPrimitiveType(true) + testNonPrimitiveType(true) } test("non-primitive type with nullability:false") { - tesNonPrimitiveType(false) - } - - def cachePrimitiveTest(data: DataFrame, dataType: String) { - data.createOrReplaceTempView(s"testData$dataType") - val useColumnBatches = true - withSQLConf(SQLConf.CACHE_CODEGEN.key -> useColumnBatches.toString) { - Seq(MEMORY_ONLY, MEMORY_ONLY_SER).map { storageLevel => - val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val inMemoryRelation = InMemoryRelation(useCompression = false, 5, storageLevel, plan, None) - - assert(inMemoryRelation.useColumnarBatches == useColumnBatches) - assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == storageLevel) - inMemoryRelation.cachedColumnBuffers.collect().head match { - case _: CachedColumnarBatch => assert(useColumnBatches) - case _: CachedBatchBytes => assert(!useColumnBatches) - case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") - } - checkAnswer(inMemoryRelation, data.collect().toSeq) - } - } + testNonPrimitiveType(false) } test("all data type w && w/o nullability") { From 3eb4ebe86db709b7eb5321e1d09d9b0ab266653f Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 2 Dec 2016 03:34:38 +0900 Subject: [PATCH 39/40] remove duplicated test --- .../sql/execution/columnar/InMemoryColumnarQuerySuite.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index fc375bc36f71..15323e7c85a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -390,12 +390,6 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val columnarIterator2 = new GenerateColumnAccessor(sparkConf).generate(columnTypes2) } - test("access only some column of the all of columns") { - val df = spark.range(1, 100).map(i => (i, (i + 1).toFloat)).toDF("i", "f").cache - df.count - assert(df.filter("f <= 10.0").count == 9) - } - test("access columns in CachedColumnarBatch without whole stage codegen") { // whole stage codegen is not applied to a row with more than WHOLESTAGE_MAX_NUM_FIELDS fields val dummySeq = Seq.range(0, 20) From b15d9d5724936f5946d99acc40b75754e8583aa6 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 20 Jan 2017 11:20:41 +0900 Subject: [PATCH 40/40] merge with master --- .../sql/execution/ColumnarBatchScan.scala | 164 ------------------ 1 file changed, 164 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 088681628cc3..05c861e353da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -1,4 +1,3 @@ -<<<<<<< fc1f6aa66c67e7d97ded9a9816ad0fc3c7439ca2 /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -150,166 +149,3 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { } } -======= -/* - * 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.SparkConf -import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector, OnHeapUnsafeColumnVector} -import org.apache.spark.sql.types.DataType - - -/** - * Helper trait for abstracting scan functionality using - * [[org.apache.spark.sql.execution.vectorized.ColumnarBatch]]es. - */ -private[sql] trait ColumnarBatchScan extends CodegenSupport { - - val columnIndexes: Array[Int] = null - - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) - - val inMemoryTableScan: InMemoryTableScanExec = null - - lazy val enableScanStatistics: Boolean = - sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean - - /** - * Generate [[ColumnVector]] expressions for our parent to consume as rows. - * This is called once per [[ColumnarBatch]]. - */ - private def genCodeColumnVector( - ctx: CodegenContext, - columnVar: String, - ordinal: String, - dataType: DataType, - nullable: Boolean): ExprCode = { - val javaType = ctx.javaType(dataType) - val value = ctx.getValue(columnVar, dataType, ordinal) - val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } - val valueVar = ctx.freshName("value") - val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { - s""" - boolean $isNullVar = $columnVar.isNullAt($ordinal); - $javaType $valueVar = $isNullVar ? ${ctx.defaultValue(dataType)} : ($value); - """ - } else { - s"$javaType $valueVar = $value;" - }).trim - ExprCode(code, isNullVar, valueVar) - } - - /** - * Produce code to process the input iterator as [[ColumnarBatch]]es. - * This produces an [[UnsafeRow]] for each row in each batch. - */ - // TODO: return ColumnarBatch.Rows instead - override protected def doProduce(ctx: CodegenContext): String = { - val input = ctx.freshName("input") - // PhysicalRDD always just has one input - ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") - - // metrics - val numOutputRows = metricTerm(ctx, "numOutputRows") - val scanTimeMetric = metricTerm(ctx, "scanTime") - val scanTimeTotalNs = ctx.freshName("scanTime") - ctx.addMutableState("long", scanTimeTotalNs, s"$scanTimeTotalNs = 0;") - val incReadBatches = if (!enableScanStatistics) "" else { - val readPartitions = ctx.addReferenceObj("readPartitions", inMemoryTableScan.readPartitions) - val readBatches = ctx.addReferenceObj("readBatches", inMemoryTableScan.readBatches) - ctx.addMutableState("int", "initializeInMemoryTableScanStatistics", - s""" - |$readPartitions.setValue(0); - |$readBatches.setValue(0); - |if ($input.hasNext()) { $readPartitions.add(1); } - """.stripMargin) - s"$readBatches.add(1);" - } - - val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" - val batch = ctx.freshName("batch") - ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") - - val generateDecompress = if (inMemoryTableScan != null) true else false - val confVar = if (!generateDecompress) null else { - val conf = inMemoryTableScan.sparkContext.conf - ctx.addReferenceObj("conf", conf, classOf[SparkConf].getName) - } - val onHeapUnsafeColumnVectorCls = classOf[OnHeapUnsafeColumnVector].getName - - val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" - val idx = ctx.freshName("batchIdx") - ctx.addMutableState("int", idx, s"$idx = 0;") - val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) - val columnAssigns = colVars.zipWithIndex.map { case (name, i) => - ctx.addMutableState(columnVectorClz, name, s"$name = null;") - val index = if (columnIndexes == null) i else columnIndexes(i) - val decompress = if (!generateDecompress) "" - else s" (($onHeapUnsafeColumnVectorCls)$name).decompress($confVar);" - s"$name = $batch.column($index);$decompress" - } - - val nextBatch = ctx.freshName("nextBatch") - ctx.addNewFunction(nextBatch, - s""" - |private void $nextBatch() throws java.io.IOException { - | long getBatchStart = System.nanoTime(); - | if ($input.hasNext()) { - | $batch = ($columnarBatchClz)$input.next(); - | $incReadBatches - | $numOutputRows.add($batch.numRows()); - | $idx = 0; - | ${columnAssigns.mkString("", "\n", "\n")} - | } - | $scanTimeTotalNs += System.nanoTime() - getBatchStart; - |}""".stripMargin) - - ctx.currentVars = null - val rowidx = ctx.freshName("rowIdx") - val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => - genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) - } - s""" - |if ($batch == null) { - | $nextBatch(); - |} - |while ($batch != null) { - | int numRows = $batch.numRows(); - | while ($idx < numRows) { - | int $rowidx = $idx++; - | ${consume(ctx, columnsBatchInput).trim} - | if (shouldStop()) return; - | } - | $batch = null; - | $nextBatch(); - |} - |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); - |$scanTimeTotalNs = 0; - """.stripMargin - } - -} ->>>>>>> support UDT and column pruning