From be1ae40a6a1c1097909006570f7ce0fa42097128 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 17 Jun 2016 13:44:18 -0700 Subject: [PATCH 01/20] Move it --- .../execution/columnar/InMemoryRelation.scala | 210 ++++++++++++++++++ .../columnar/InMemoryTableScanExec.scala | 186 +--------------- 2 files changed, 211 insertions(+), 185 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala 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 new file mode 100644 index 000000000000..c546d4bc979c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -0,0 +1,210 @@ +/* + * 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 scala.collection.JavaConverters._ + +import org.apache.commons.lang.StringUtils + +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.CollectionAccumulator + + +private[sql] object InMemoryRelation { + def apply( + useCompression: Boolean, + batchSize: Int, + storageLevel: StorageLevel, + child: SparkPlan, + tableName: Option[String]): InMemoryRelation = + new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)() +} + + +/** + * CachedBatch is a cached batch of rows. + * + * @param numRows The total number of rows in this batch + * @param buffers The buffers for serialized columns + * @param stats The stat of columns + */ +private[columnar] +case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + +private[sql] case class InMemoryRelation( + output: Seq[Attribute], + useCompression: Boolean, + batchSize: Int, + storageLevel: StorageLevel, + @transient child: SparkPlan, + tableName: Option[String])( + @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, + @transient private[sql] var _statistics: Statistics = null, + private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) + extends logical.LeafNode with MultiInstanceRelation { + + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) + + override def producedAttributes: AttributeSet = outputSet + + private[sql] val batchStats: CollectionAccumulator[InternalRow] = + if (_batchStats == null) { + child.sqlContext.sparkContext.collectionAccumulator[InternalRow] + } else { + _batchStats + } + + @transient val partitionStatistics = new PartitionStatistics(output) + + private def computeSizeInBytes = { + val sizeOfRow: Expression = + BindReferences.bindReference( + output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), + partitionStatistics.schema) + + batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + } + + // Statistics propagation contracts: + // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data + // 2. Only propagate statistics when `_statistics` is non-null + private def statisticsToBePropagated = if (_statistics == null) { + val updatedStats = statistics + if (_statistics == null) null else updatedStats + } else { + _statistics + } + + override def statistics: Statistics = { + if (_statistics == null) { + if (batchStats.value.isEmpty) { + // Underlying columnar RDD hasn't been materialized, no useful statistics information + // available, return the default statistics. + Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) + } else { + // Underlying columnar RDD has been materialized, required information has also been + // collected via the `batchStats` accumulator, compute the final statistics, + // and update `_statistics`. + _statistics = Statistics(sizeInBytes = computeSizeInBytes) + _statistics + } + } else { + // Pre-computed statistics + _statistics + } + } + + // 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) { + buildBuffers() + } + + def recache(): Unit = { + _cachedColumnBuffers.unpersist() + _cachedColumnBuffers = null + buildBuffers() + } + + private def buildBuffers(): Unit = { + val output = child.output + val cached = child.execute().mapPartitionsInternal { rowIterator => + new Iterator[CachedBatch] { + def next(): CachedBatch = { + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext && rowCount < batchSize + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + val row = rowIterator.next() + + // Added for SPARK-6082. This assertion can be useful for scenarios when something + // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM + // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat + // hard to decipher. + assert( + row.numFields == columnBuilders.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) { + columnBuilders(i).appendFrom(row, i) + totalSize += columnBuilders(i).columnStats.sizeInBytes + i += 1 + } + rowCount += 1 + } + + val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) + .flatMap(_.values)) + + batchStats.add(stats) + CachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, stats) + } + + 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)( + _cachedColumnBuffers, statisticsToBePropagated, batchStats) + } + + override def newInstance(): this.type = { + new InMemoryRelation( + output.map(_.newInstance()), + useCompression, + batchSize, + storageLevel, + child, + tableName)( + _cachedColumnBuffers, + statisticsToBePropagated, + batchStats).asInstanceOf[this.type] + } + + def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers + + override protected def otherCopyArgs: Seq[AnyRef] = + Seq(_cachedColumnBuffers, statisticsToBePropagated, 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 ce630bcbca4c..2695f356cd3e 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 @@ -17,200 +17,16 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.JavaConverters._ - -import org.apache.commons.lang.StringUtils - -import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{AccumulatorContext, CollectionAccumulator, LongAccumulator} - - -private[sql] object InMemoryRelation { - def apply( - useCompression: Boolean, - batchSize: Int, - storageLevel: StorageLevel, - child: SparkPlan, - tableName: Option[String]): InMemoryRelation = - new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)() -} - -/** - * CachedBatch is a cached batch of rows. - * - * @param numRows The total number of rows in this batch - * @param buffers The buffers for serialized columns - * @param stats The stat of columns - */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) - -private[sql] case class InMemoryRelation( - output: Seq[Attribute], - useCompression: Boolean, - batchSize: Int, - storageLevel: StorageLevel, - @transient child: SparkPlan, - tableName: Option[String])( - @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - @transient private[sql] var _statistics: Statistics = null, - private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) - extends logical.LeafNode with MultiInstanceRelation { - - override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) - - override def producedAttributes: AttributeSet = outputSet - - private[sql] val batchStats: CollectionAccumulator[InternalRow] = - if (_batchStats == null) { - child.sqlContext.sparkContext.collectionAccumulator[InternalRow] - } else { - _batchStats - } - - @transient val partitionStatistics = new PartitionStatistics(output) - - private def computeSizeInBytes = { - val sizeOfRow: Expression = - BindReferences.bindReference( - output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), - partitionStatistics.schema) - - batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum - } - - // Statistics propagation contracts: - // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data - // 2. Only propagate statistics when `_statistics` is non-null - private def statisticsToBePropagated = if (_statistics == null) { - val updatedStats = statistics - if (_statistics == null) null else updatedStats - } else { - _statistics - } - - override def statistics: Statistics = { - if (_statistics == null) { - if (batchStats.value.isEmpty) { - // Underlying columnar RDD hasn't been materialized, no useful statistics information - // available, return the default statistics. - Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) - } else { - // Underlying columnar RDD has been materialized, required information has also been - // collected via the `batchStats` accumulator, compute the final statistics, - // and update `_statistics`. - _statistics = Statistics(sizeInBytes = computeSizeInBytes) - _statistics - } - } else { - // Pre-computed statistics - _statistics - } - } - - // 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) { - buildBuffers() - } - def recache(): Unit = { - _cachedColumnBuffers.unpersist() - _cachedColumnBuffers = null - buildBuffers() - } - - private def buildBuffers(): Unit = { - val output = child.output - val cached = child.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { - val columnBuilders = output.map { attribute => - ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) - }.toArray - - var rowCount = 0 - var totalSize = 0L - while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { - val row = rowIterator.next() - - // Added for SPARK-6082. This assertion can be useful for scenarios when something - // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM - // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat - // hard to decipher. - assert( - row.numFields == columnBuilders.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) { - columnBuilders(i).appendFrom(row, i) - totalSize += columnBuilders(i).columnStats.sizeInBytes - i += 1 - } - rowCount += 1 - } - - val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) - .flatMap(_.values)) - - batchStats.add(stats) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - - 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)( - _cachedColumnBuffers, statisticsToBePropagated, batchStats) - } - - override def newInstance(): this.type = { - new InMemoryRelation( - output.map(_.newInstance()), - useCompression, - batchSize, - storageLevel, - child, - tableName)( - _cachedColumnBuffers, - statisticsToBePropagated, - batchStats).asInstanceOf[this.type] - } - - def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers - - override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) -} private[sql] case class InMemoryTableScanExec( attributes: Seq[Attribute], From bf11d278cb6420c10d4f748e2b19cead4a3f6391 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 17 Jun 2016 14:47:20 -0700 Subject: [PATCH 02/20] Add benchmark code --- .../execution/columnar/InMemoryRelation.scala | 45 ++++++++++++++++++ .../benchmark/AggregateBenchmark.scala | 47 +++++++++++++------ 2 files changed, 78 insertions(+), 14 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 c546d4bc979c..d31556edb4a4 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 @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.commons.lang.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.InternalRow @@ -30,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.vectorized.ColumnVector import org.apache.spark.storage.StorageLevel import org.apache.spark.util.CollectionAccumulator @@ -184,6 +186,49 @@ private[sql] 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 bf3a39c84b3b..be0413c6d55d 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 @@ -130,31 +130,50 @@ 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 => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + println("Sleeping for 5 seconds before starting...") + Thread.sleep(5000L) + + benchmark.addCase(s"codegen = F hashmap = F cache = F", numIters) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) - f() + run() } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 3) - f() - } +// benchmark.addCase(s"codegen = F hashmap = F cache = T compress = F", numIters) { iter => +// sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) +// sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) +// sparkSession.conf.set("spark.sql.inMemoryColumnarStorage.compressed", value = false) +// run(cache = true) +// } + +// benchmark.addCase(s"codegen = F hashmap = F cache = T compress = T", numIters) { iter => +// sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) +// sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) +// sparkSession.conf.set("spark.sql.inMemoryColumnarStorage.compressed", value = true) +// run(cache = true) +// } benchmark.run() From 82499c37f8a2a539e97febc05f3f416411dc0985 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 20 Jun 2016 11:35:21 -0700 Subject: [PATCH 03/20] backup --- .../execution/vectorized/ColumnVector.java | 11 +- .../vectorized/OnHeapColumnVector.java | 26 +++ .../execution/columnar/InMemoryRelation.scala | 95 ++++++----- .../columnar/InMemoryTableScanExec.scala | 158 ++++-------------- .../org/apache/spark/sql/QueryTest.scala | 1 + .../benchmark/AggregateBenchmark.scala | 18 +- 6 files changed, 133 insertions(+), 176 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 3f9425525669..a87f9d1cd5f5 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; @@ -56,7 +59,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 @@ -849,7 +856,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 7fb7617050f2..92ea80f2fa69 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 d31556edb4a4..8028e038152c 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 @@ -57,6 +57,9 @@ private[sql] object InMemoryRelation { private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +private[columnar] +case class ColumnarCachedBatch(numRows: Int, buffers: Array[Array[Long]]) + private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, @@ -65,6 +68,7 @@ private[sql] case class InMemoryRelation( @transient child: SparkPlan, tableName: Option[String])( @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, + @transient private[sql] var _cachedColumnVectors: RDD[ColumnarCachedBatch] = null, @transient private[sql] var _statistics: Statistics = null, private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { @@ -124,12 +128,14 @@ private[sql] 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 = { @@ -186,53 +192,54 @@ private[sql] 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( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated, batchStats) + _cachedColumnBuffers, _cachedColumnVectors, statisticsToBePropagated, batchStats) } override def newInstance(): this.type = { @@ -244,11 +251,13 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, + _cachedColumnVectors, statisticsToBePropagated, batchStats).asInstanceOf[this.type] } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers + def cachedColumnVectors: RDD[ColumnarCachedBatch] = _cachedColumnVectors override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, statisticsToBePropagated, 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 2695f356cd3e..7bbd00ea2a22 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 private[sql] case class InMemoryTableScanExec( @@ -36,9 +34,6 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren - private[sql] override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - override def output: Seq[Attribute] = attributes // The cached version does not change the outputPartitioning of the original SparkPlan. @@ -47,132 +42,51 @@ private[sql] case class InMemoryTableScanExec( // The cached version does not change the outputOrdering of the original SparkPlan. override def outputOrdering: Seq[SortOrder] = relation.child.outputOrdering - private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a) - - // Returned filter predicate should return false iff it is impossible for the input expression - // to evaluate to `true' based on statistics collected about this partition batch. - @transient val buildFilter: PartialFunction[Expression, Expression] = { - case And(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => - (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) - - case Or(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => - buildFilter(lhs) || buildFilter(rhs) - - case EqualTo(a: AttributeReference, l: Literal) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualTo(l: Literal, a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l - case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound - - case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l - case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound - - case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound - case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l - - case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound - case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l - - case IsNull(a: Attribute) => statsFor(a).nullCount > 0 - case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 - } - - val partitionFilters: Seq[Expression] = { - predicates.flatMap { p => - val filter = buildFilter.lift(p) - val boundFilter = - filter.map( - BindReferences.bindReference( - _, - relation.partitionStatistics.schema, - allowFailures = true)) - - boundFilter.foreach(_ => - filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) - - // If the filter can't be resolved then we are missing required statistics. - boundFilter.filter(_.resolved) - } - } - - lazy val enableAccumulators: Boolean = - sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean - // Accumulators used for testing purposes 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") - - if (enableAccumulators) { - readPartitions.setValue(0) - readBatches.setValue(0) - } - - // Using these variables here to avoid serialization of entire objects (if referenced directly) - // within the map Partitions closure. - val schema = relation.partitionStatistics.schema - val schemaIndex = schema.zipWithIndex - val relOutput: AttributeSeq = relation.output - val buffers = relation.cachedColumnBuffers - - buffers.mapPartitionsInternal { cachedBatchIterator => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip - - // Do partition batch pruning if enabled - val cachedBatchesToScan = - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter(cachedBatch.stats)) { - def statsString: String = schemaIndex.map { - case (a, i) => - val value = cachedBatch.stats.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - logInfo(s"Skipping partition based on stats $statsString") - false - } else { - if (enableAccumulators) { - readBatches.add(1) - } - true + 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 } - } else { - cachedBatchIterator } - // update SQL metrics - val withMetrics = cachedBatchesToScan.map { batch => - numOutputRows += batch.numRows - 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) + 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 + } } - columnarIterator } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 742f036e5503..5f2181b49e0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -323,6 +323,7 @@ abstract class QueryTest extends PlanTest { origin.child, l.tableName)( origin.cachedColumnBuffers, + origin.cachedColumnVectors, l._statistics, origin._batchStats) case p => 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 be0413c6d55d..c38a7d8a2ecf 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 @@ -155,19 +155,19 @@ class AggregateBenchmark extends BenchmarkBase { println("Sleeping for 5 seconds before starting...") Thread.sleep(5000L) - benchmark.addCase(s"codegen = F hashmap = F cache = F", numIters) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) - run() - } - -// benchmark.addCase(s"codegen = F hashmap = F cache = T compress = F", numIters) { iter => +// benchmark.addCase(s"codegen = F hashmap = F cache = F", numIters) { iter => // sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) // sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) -// sparkSession.conf.set("spark.sql.inMemoryColumnarStorage.compressed", value = false) -// run(cache = true) +// run() // } + benchmark.addCase(s"codegen = F hashmap = F cache = T compress = F", numIters) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) + sparkSession.conf.set("spark.sql.inMemoryColumnarStorage.compressed", value = false) + run(cache = true) + } + // benchmark.addCase(s"codegen = F hashmap = F cache = T compress = T", numIters) { iter => // sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) // sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) From 2f12e96f3d23d49587e15364861dbe34bdfc8972 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 20 Jun 2016 15:36:44 -0700 Subject: [PATCH 04/20] Narrow benchmarked code + add back old scan code --- .../org/apache/spark/util/Benchmark.scala | 17 +- .../vectorized/OnHeapColumnVector.java | 44 ++-- .../execution/columnar/InMemoryRelation.scala | 27 +-- .../columnar/InMemoryTableScanExec.scala | 203 ++++++++++++++---- .../benchmark/AggregateBenchmark.scala | 75 +++---- 5 files changed, 245 insertions(+), 121 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 92ea80f2fa69..3014d27bef03 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 8028e038152c..cf281d3ffb25 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 @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel import org.apache.spark.util.CollectionAccumulator @@ -58,7 +59,7 @@ private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) private[columnar] -case class ColumnarCachedBatch(numRows: Int, buffers: Array[Array[Long]]) +case class ColumnarCachedBatch(numRows: Int, buffers: ColumnarBatch) private[sql] case class InMemoryRelation( output: Seq[Attribute], @@ -192,38 +193,32 @@ private[sql] 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 7bbd00ea2a22..1e44cfd82bbe 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 private[sql] case class InMemoryTableScanExec( @@ -34,6 +36,9 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren + private[sql] override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + override def output: Seq[Attribute] = attributes // The cached version does not change the outputPartitioning of the original SparkPlan. @@ -42,51 +47,177 @@ private[sql] case class InMemoryTableScanExec( // The cached version does not change the outputOrdering of the original SparkPlan. override def outputOrdering: Seq[SortOrder] = relation.child.outputOrdering + private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a) + + // Returned filter predicate should return false iff it is impossible for the input expression + // to evaluate to `true' based on statistics collected about this partition batch. + @transient val buildFilter: PartialFunction[Expression, Expression] = { + case And(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => + (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) + + case Or(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) || buildFilter(rhs) + + case EqualTo(a: AttributeReference, l: Literal) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualTo(l: Literal, a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l + case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound + + case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l + case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound + + case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound + case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound + case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l + + case IsNull(a: Attribute) => statsFor(a).nullCount > 0 + case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 + } + + val partitionFilters: Seq[Expression] = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + relation.partitionStatistics.schema, + allowFailures = true)) + + boundFilter.foreach(_ => + filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) + + // If the filter can't be resolved then we are missing required statistics. + boundFilter.filter(_.resolved) + } + } + + lazy val enableAccumulators: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + // Accumulators used for testing purposes lazy val readPartitions = sparkContext.longAccumulator lazy val readBatches = sparkContext.longAccumulator + private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning + 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 + val numOutputRows = longMetric("numOutputRows") + + if (enableAccumulators) { + readPartitions.setValue(0) + readBatches.setValue(0) + } + + // Using these variables here to avoid serialization of entire objects (if referenced directly) + // within the map Partitions closure. + val schema = relation.partitionStatistics.schema + val schemaIndex = schema.zipWithIndex + val relOutput: AttributeSeq = relation.output + val buffers = relation.cachedColumnBuffers + + buffers.mapPartitionsInternal { cachedBatchIterator => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + attributes.map { a => + relOutput.indexOf(a.exprId) -> a.dataType + }.unzip + + // Do partition batch pruning if enabled + val cachedBatchesToScan = + if (inMemoryPartitionPruningEnabled) { + cachedBatchIterator.filter { cachedBatch => + if (!partitionFilter(cachedBatch.stats)) { + def statsString: String = schemaIndex.map { + case (a, i) => + val value = cachedBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + logInfo(s"Skipping partition based on stats $statsString") + false + } else { + if (enableAccumulators) { + readBatches.add(1) + } + true } - hasNext - } else { - true // currentBatch != null } + } else { + cachedBatchIterator } - 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 - } + // update SQL metrics + val withMetrics = cachedBatchesToScan.map { batch => + numOutputRows += batch.numRows + 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) + } + 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 c38a7d8a2ecf..0589f0898478 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,62 +131,51 @@ 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() } } - println("Sleeping for 5 seconds before starting...") - Thread.sleep(5000L) + runBenchmark("codegen = F hashmap = F cache = F", cache = false, Map( + "spark.sql.codegen.wholeStage" -> "false", + "spark.sql.codegen.aggregate.map.columns.max" -> "0" + )) -// benchmark.addCase(s"codegen = F hashmap = F cache = F", numIters) { iter => -// sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) -// sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) -// run() -// } + runBenchmark("codegen = F hashmap = F cache = T compress = F", cache = true, Map( + "spark.sql.codegen.wholeStage" -> "false", + "spark.sql.codegen.aggregate.map.columns.max" -> "0", + "spark.sql.inMemoryColumnarStorage.compressed" -> "false" + )) - benchmark.addCase(s"codegen = F hashmap = F cache = T compress = F", numIters) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) - sparkSession.conf.set("spark.sql.inMemoryColumnarStorage.compressed", value = false) - run(cache = true) - } - -// benchmark.addCase(s"codegen = F hashmap = F cache = T compress = T", numIters) { iter => -// sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) -// sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) -// sparkSession.conf.set("spark.sql.inMemoryColumnarStorage.compressed", value = true) -// run(cache = true) -// } + runBenchmark("codegen = F hashmap = F cache = T compress = T", cache = true, Map( + "spark.sql.codegen.wholeStage" -> "false", + "spark.sql.codegen.aggregate.map.columns.max" -> "0", + "spark.sql.inMemoryColumnarStorage.compressed" -> "true" + )) 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 6da1e71be250fd4ddfe5cbca076ede3b78d67d0e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Jun 2016 11:28:19 -0700 Subject: [PATCH 05/20] Fix benchmark to time only the read path --- .../spark/scheduler/TaskSetManager.scala | 2 +- .../org/apache/spark/util/Benchmark.scala | 28 +++++++++------ .../benchmark/AggregateBenchmark.scala | 35 ++++++++++++------- .../execution/benchmark/BenchmarkBase.scala | 4 ++- 4 files changed, 44 insertions(+), 25 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 2eedd201ca35..f55b54259b54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -660,7 +660,7 @@ private[spark] class TaskSetManager( reason.asInstanceOf[TaskFailedReason].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 0589f0898478..1a0fecc298a8 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,48 +132,59 @@ 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) } } - runBenchmark("codegen = F hashmap = F cache = F", cache = false, Map( + addBenchmark("codegen = F hashmap = F cache = F", cache = false, Map( "spark.sql.codegen.wholeStage" -> "false", "spark.sql.codegen.aggregate.map.columns.max" -> "0" )) - runBenchmark("codegen = F hashmap = F cache = T compress = F", cache = true, Map( + addBenchmark("codegen = F hashmap = F cache = T compress = F", cache = true, Map( "spark.sql.codegen.wholeStage" -> "false", "spark.sql.codegen.aggregate.map.columns.max" -> "0", "spark.sql.inMemoryColumnarStorage.compressed" -> "false" )) - runBenchmark("codegen = F hashmap = F cache = T compress = T", cache = true, Map( - "spark.sql.codegen.wholeStage" -> "false", - "spark.sql.codegen.aggregate.map.columns.max" -> "0", - "spark.sql.inMemoryColumnarStorage.compressed" -> "true" - )) +// addBenchmark("codegen = F hashmap = F cache = T compress = T", cache = true, Map( +// "spark.sql.codegen.wholeStage" -> "false", +// "spark.sql.codegen.aggregate.map.columns.max" -> "0", +// "spark.sql.inMemoryColumnarStorage.compressed" -> "true" +// )) benchmark.run() } 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 fdf321e3c6d9c057193620bfba8fbc97a01e8513 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Jun 2016 14:35:01 -0700 Subject: [PATCH 06/20] 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 | 21 +++-- .../columnar/InMemoryTableScanExec.scala | 81 +++++++++---------- .../benchmark/AggregateBenchmark.scala | 34 +++++--- 3 files changed, 68 insertions(+), 68 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 cf281d3ffb25..0fe01fbfa519 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.vectorized.{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.CollectionAccumulator @@ -58,9 +58,6 @@ private[sql] object InMemoryRelation { private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) -private[columnar] -case class ColumnarCachedBatch(numRows: Int, buffers: ColumnarBatch) - private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, @@ -69,7 +66,7 @@ private[sql] case class InMemoryRelation( @transient child: SparkPlan, tableName: Option[String])( @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - @transient private[sql] var _cachedColumnVectors: RDD[ColumnarCachedBatch] = null, + @transient private[sql] var _cachedColumnVectors: RDD[ColumnarBatch] = null, @transient private[sql] var _statistics: Statistics = null, private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { @@ -197,9 +194,9 @@ private[sql] 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 @@ -211,14 +208,14 @@ private[sql] 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 @@ -252,7 +249,7 @@ private[sql] 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, statisticsToBePropagated, 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 1e44cfd82bbe..59a0271873d0 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 @@ -108,7 +109,7 @@ private[sql] 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) { @@ -176,48 +177,40 @@ private[sql] 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 1a0fecc298a8..9718ebdc49f5 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 @@ -132,31 +133,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") } @@ -169,14 +174,19 @@ class AggregateBenchmark extends BenchmarkBase { benchmark.addCase(name, numIters, prepare, cleanup) { _ => doCollect(ds) } } - addBenchmark("codegen = F hashmap = F cache = F", cache = false, Map( - "spark.sql.codegen.wholeStage" -> "false", - "spark.sql.codegen.aggregate.map.columns.max" -> "0" + // ALL OF THESE ARE codegen = F, hashmap = F + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") + + addBenchmark("cache = F", cache = false) + + addBenchmark("cache = T columnar = F compress = F", cache = true, Map( + "spark.sql.inMemoryColumnarScan" -> "false", + "spark.sql.inMemoryColumnarStorage.compressed" -> "false" )) - addBenchmark("codegen = F hashmap = F cache = T compress = F", cache = true, Map( - "spark.sql.codegen.wholeStage" -> "false", - "spark.sql.codegen.aggregate.map.columns.max" -> "0", + addBenchmark("cache = T columnar = T compress = F", cache = true, Map( + "spark.sql.inMemoryColumnarScan" -> "true", "spark.sql.inMemoryColumnarStorage.compressed" -> "false" )) From d0d2661f47d351dab0627fde44e192e144e661a6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Jun 2016 17:45:34 -0700 Subject: [PATCH 07/20] Always enable codegen and vectorized hashmap --- .../sql/execution/columnar/InMemoryRelation.scala | 9 ++++----- .../sql/execution/benchmark/AggregateBenchmark.scala | 11 +++++------ 2 files changed, 9 insertions(+), 11 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 0fe01fbfa519..83071c337559 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 @@ -126,14 +126,14 @@ private[sql] 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 = { @@ -190,8 +190,7 @@ private[sql] 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] { @@ -208,7 +207,7 @@ private[sql] 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 } 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 9718ebdc49f5..425ee7ca507e 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 @@ -174,9 +174,9 @@ class AggregateBenchmark extends BenchmarkBase { benchmark.addCase(name, numIters, prepare, cleanup) { _ => doCollect(ds) } } - // ALL OF THESE ARE codegen = F, hashmap = F - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") + // 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", "100") addBenchmark("cache = F", cache = false) @@ -190,9 +190,8 @@ class AggregateBenchmark extends BenchmarkBase { "spark.sql.inMemoryColumnarStorage.compressed" -> "false" )) -// addBenchmark("codegen = F hashmap = F cache = T compress = T", cache = true, Map( -// "spark.sql.codegen.wholeStage" -> "false", -// "spark.sql.codegen.aggregate.map.columns.max" -> "0", +// addBenchmark("cache = T columnar = F compress = T", cache = true, Map( +// "spark.sql.inMemoryColumnarScan" -> "false", // "spark.sql.inMemoryColumnarStorage.compressed" -> "true" // )) From 570d0c3470bfcd095c4a0389cd05c1a2c764bd25 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 12:48:37 -0700 Subject: [PATCH 08/20] 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 425ee7ca507e..0f0c38da3b50 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,14 +132,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. @@ -160,7 +160,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 3e96f4efbe17a1f7f6047d937379401daa6f252c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 14:57:56 -0700 Subject: [PATCH 09/20] 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 59a0271873d0..8c4531897e70 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} private[sql] 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 @@ -203,14 +204,105 @@ private[sql] 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 0f0c38da3b50..94aef5e3de6e 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 @@ -135,11 +135,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. @@ -160,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 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 5726d11adb202136f827133ce8f9a3ab595a17f0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 16:10:41 -0700 Subject: [PATCH 10/20] 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 a87f9d1cd5f5..02b8f01af322 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; @@ -59,10 +56,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. @@ -856,7 +850,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 3014d27bef03..7fb7617050f2 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 83071c337559..6ddd2ed2c5db 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 @@ -136,6 +136,9 @@ private[sql] 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 => @@ -190,6 +193,9 @@ private[sql] 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 8c4531897e70..c63c3d0e274f 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 @@ private[sql] 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 private[sql] override lazy val metrics = Map( @@ -110,7 +121,8 @@ private[sql] 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) { @@ -178,81 +190,10 @@ private[sql] 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 @@ -305,4 +246,30 @@ private[sql] 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 94aef5e3de6e..b97f0c5939bf 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,7 +158,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) @@ -196,6 +196,16 @@ class AggregateBenchmark extends BenchmarkBase { // "spark.sql.inMemoryColumnarStorage.compressed" -> "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 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 f4f81826b5facb83e1ab6cd0988d056feedc5d54 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 22 Jun 2016 16:19:42 -0700 Subject: [PATCH 11/20] 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 02b8f01af322..3f9425525669 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 @@ -57,7 +57,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 6ddd2ed2c5db..67b60aec92f9 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 @@ -126,12 +126,17 @@ private[sql] 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() } @@ -213,6 +218,7 @@ private[sql] 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 c63c3d0e274f..e6fecaae28b8 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} @@ -121,7 +121,7 @@ private[sql] 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") @@ -192,7 +192,9 @@ private[sql] 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") @@ -248,7 +250,9 @@ private[sql] 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 41d52b75fa39d09adb40a792cef4e2ffe2e0851f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 15:10:41 -0700 Subject: [PATCH 12/20] 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 | 6 +- .../columnar/GenerateColumnarBatch.scala | 133 ++++++++++++++++++ .../execution/columnar/InMemoryRelation.scala | 68 ++++----- .../columnar/InMemoryTableScanExec.scala | 12 +- 4 files changed, 171 insertions(+), 48 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 de2503a87ab7..3e57fcc4b883 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 @@ -69,7 +69,7 @@ private[sql] class CacheManager extends Logging { /** Clears all cached tables. */ private[sql] def clearCache(): Unit = writeLock { - cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) + cachedData.foreach(_.cachedRepresentation.unpersist()) cachedData.clear() } @@ -113,7 +113,7 @@ private[sql] 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 @@ -167,7 +167,7 @@ private[sql] 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 67b60aec92f9..57e4d9e075ce 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 @@ -21,7 +21,6 @@ import scala.collection.JavaConverters._ import org.apache.commons.lang.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.InternalRow @@ -71,6 +70,14 @@ private[sql] case class InMemoryRelation( private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) 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[QueryPlan[_]] = Seq(child) override def producedAttributes: AttributeSet = outputSet @@ -124,21 +131,32 @@ private[sql] 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() + } } /** @@ -200,39 +218,13 @@ private[sql] 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 e6fecaae28b8..773706efcefe 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 @@ private[sql] 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]. @@ -122,7 +119,8 @@ private[sql] 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 06bbfdbf040e509b88e8462c80bb566e0ac314c8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 16:01:44 -0700 Subject: [PATCH 13/20] Move cache benchmark to new file --- .../spark/scheduler/TaskSetManager.scala | 2 +- .../benchmark/AggregateBenchmark.scala | 93 +++++--------- .../execution/benchmark/BenchmarkBase.scala | 4 +- .../execution/benchmark/CacheBenchmark.scala | 113 ++++++++++++++++++ 4 files changed, 142 insertions(+), 70 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 f55b54259b54..2eedd201ca35 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -660,7 +660,7 @@ private[spark] class TaskSetManager( reason.asInstanceOf[TaskFailedReason].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 b97f0c5939bf..bf3a39c84b3b 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 @@ -131,82 +130,44 @@ 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() } - // 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", "100") - - addBenchmark("cache = F", cache = false) - - addBenchmark("cache = T columnar = F compress = F", cache = true, Map( - "spark.sql.inMemoryColumnarScan" -> "false", - "spark.sql.inMemoryColumnarStorage.compressed" -> "false" - )) + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) + f() + } - addBenchmark("cache = T columnar = T compress = F", cache = true, Map( - "spark.sql.inMemoryColumnarScan" -> "true", - "spark.sql.inMemoryColumnarStorage.compressed" -> "false" - )) + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 3) + f() + } -// addBenchmark("cache = T columnar = F compress = T", cache = true, Map( -// "spark.sql.inMemoryColumnarScan" -> "false", -// "spark.sql.inMemoryColumnarStorage.compressed" -> "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 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 1a12d06e4e3f71cd21229d9adc766d5643dfdfa3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 16:43:13 -0700 Subject: [PATCH 14/20] Abstract codegen code into ColumnarBatchScan --- .../sql/execution/ColumnarBatchScan.scala | 129 ++++++++++++++++++ .../spark/sql/execution/ExistingRDD.scala | 92 +------------ .../columnar/InMemoryTableScanExec.scala | 97 +------------ 3 files changed, 135 insertions(+), 183 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala 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 new file mode 100644 index 000000000000..32ac4dde12bf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -0,0 +1,129 @@ +/* + * 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.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 { + + private[sql] override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + + /** + * 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. + */ + 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 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(); + | $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 + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index e2c23a4ba867..8489179521e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution import org.apache.commons.lang.StringUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources.HadoopFsRelation @@ -229,11 +229,7 @@ private[sql] case class BatchedDataSourceScanExec( override val outputPartitioning: Partitioning, override val metadata: Map[String, String], override val metastoreTableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec with CodegenSupport { - - private[sql] override lazy val metrics = - Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + extends DataSourceScanExec with ColumnarBatchScan { protected override def doExecute(): RDD[InternalRow] = { // in the case of fallback, this batched scan should never fail because of: @@ -253,88 +249,6 @@ private[sql] case class BatchedDataSourceScanExec( override def inputRDDs(): Seq[RDD[InternalRow]] = { rdd :: Nil } - - 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. - 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 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(); - | $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 - } } private[sql] object DataSourceScanExec { 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 773706efcefe..1a7381b149ff 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 private[sql] 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 @@ -46,9 +44,6 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren - private[sql] override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - override def output: Seq[Attribute] = attributes // The cached version does not change the outputPartitioning of the original SparkPlan. @@ -188,90 +183,4 @@ private[sql] 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 8cdbdd0c729936d731e531ee10c2ba4e72ceec57 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:04:09 -0700 Subject: [PATCH 15/20] Introduce CACHE_CODEGEN config to reduce dup code --- .../columnar/GenerateColumnarBatch.scala | 1 + .../execution/columnar/InMemoryRelation.scala | 32 ++++++++++--------- .../columnar/InMemoryTableScanExec.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 7 ++++ .../org/apache/spark/sql/QueryTest.scala | 2 +- .../execution/benchmark/CacheBenchmark.scala | 15 +++++---- 6 files changed, 35 insertions(+), 24 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 902ab2cb7374..b04cab35a05c 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 @@ -31,6 +31,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.CollectionAccumulator @@ -64,16 +65,17 @@ private[sql] case class InMemoryRelation( storageLevel: StorageLevel, @transient child: SparkPlan, tableName: Option[String])( + // Note: Only one of these two RDDs will be set at any given time. + // The ColumnarBatch one is generally faster because it uses generated code. @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - @transient private[sql] var _cachedColumnVectors: RDD[ColumnarBatch] = null, + @transient private[sql] var _cachedColumnBatches: RDD[ColumnarBatch] = null, private[sql] val batchStats: CollectionAccumulator[InternalRow] = child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) 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 } @@ -105,8 +107,8 @@ private[sql] 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) { @@ -118,16 +120,16 @@ private[sql] 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() } @@ -194,7 +196,7 @@ private[sql] 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) @@ -202,13 +204,13 @@ private[sql] 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 = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, _cachedColumnVectors, batchStats) + _cachedColumnBuffers, _cachedColumnBatches, batchStats) } override def newInstance(): this.type = { @@ -220,12 +222,12 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - _cachedColumnVectors, + _cachedColumnBatches, batchStats).asInstanceOf[this.type] } 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 1a7381b149ff..29246cb48b52 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 @@ private[sql] 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 1a9bb6a0b54e..4732f4f4d10f 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 @@ -91,6 +91,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/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index cfaca157c544..73aaf89dab83 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -323,7 +323,7 @@ abstract class QueryTest extends PlanTest { origin.child, l.tableName)( origin.cachedColumnBuffers, - origin.cachedColumnVectors, + origin.cachedColumnBatches, origin.batchStats) case p => p.transformExpressions { 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 faa6776b92a8ca5281699df3af1f1fc59aa786e8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:34:57 -0700 Subject: [PATCH 16/20] Add some tests for InMemoryRelation --- .../columnar/InMemoryTableScanExec.scala | 11 +++-- .../columnar/InMemoryColumnarQuerySuite.scala | 43 +++++++++++++++++++ 2 files changed, 50 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 29246cb48b52..27474dce673c 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 @@ private[sql] 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 af3ed14c122d..798f15289719 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.{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._ @@ -231,4 +232,46 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) } + + 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 2ba6b1e2f79a1c41b56e51a7d3a01b06417f03dd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:40:11 -0700 Subject: [PATCH 17/20] Add some tests for InMemoryRelation --- .../columnar/InMemoryColumnarQuerySuite.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 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 798f15289719..120ad41e9325 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,13 +20,14 @@ 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.{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._ import org.apache.spark.sql.types._ -import org.apache.spark.storage.StorageLevel.MEMORY_ONLY +import org.apache.spark.storage.StorageLevel.{MEMORY_ONLY, NONE} class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -271,6 +272,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 7f09753a5df4465d1e4f0d57d06b53b4637f7470 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 23 Jun 2016 17:44:21 -0700 Subject: [PATCH 18/20] 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 b04cab35a05c..b3d08234a503 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 @@ -230,5 +230,5 @@ private[sql] case class InMemoryRelation( def cachedColumnBatches: RDD[ColumnarBatch] = _cachedColumnBatches override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, batchStats) + Seq(_cachedColumnBuffers, _cachedColumnBatches, batchStats) } From c72c085b32179113e546fb0251032e95106b2cd3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 24 Jun 2016 12:00:37 -0700 Subject: [PATCH 19/20] Clean up code: abstract CachedBatch and ColumnarBatch --- .../spark/sql/execution/CacheManager.scala | 6 +- .../sql/execution/ColumnarBatchScan.scala | 1 + .../columnar/GenerateColumnAccessor.scala | 3 +- .../execution/columnar/InMemoryRelation.scala | 151 ++++++++++-------- .../columnar/InMemoryTableScanExec.scala | 15 +- .../org/apache/spark/sql/QueryTest.scala | 1 - .../columnar/InMemoryColumnarQuerySuite.scala | 29 +--- 7 files changed, 108 insertions(+), 98 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 3e57fcc4b883..de2503a87ab7 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 @@ -69,7 +69,7 @@ private[sql] class CacheManager extends Logging { /** Clears all cached tables. */ private[sql] def clearCache(): Unit = writeLock { - cachedData.foreach(_.cachedRepresentation.unpersist()) + cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) cachedData.clear() } @@ -113,7 +113,7 @@ private[sql] 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 @@ -167,7 +167,7 @@ private[sql] 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/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 32ac4dde12bf..a3fc55cf44c1 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 @@ -64,6 +64,7 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { * 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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 7a14879b8b9d..e228bbc257b6 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,7 +203,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 b3d08234a503..2b54f6a0262d 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,6 +37,33 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.CollectionAccumulator +/** + * 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 + + private[sql] object InMemoryRelation { def apply( useCompression: Boolean, @@ -49,15 +76,11 @@ private[sql] 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) - private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, @@ -65,17 +88,18 @@ private[sql] case class InMemoryRelation( storageLevel: StorageLevel, @transient child: SparkPlan, tableName: Option[String])( - // Note: Only one of these two RDDs will be set at any given time. - // The ColumnarBatch one is generally faster because it uses generated code. @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - @transient private[sql] var _cachedColumnBatches: RDD[ColumnarBatch] = null, private[sql] val batchStats: CollectionAccumulator[InternalRow] = child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) 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 } @@ -105,44 +129,33 @@ private[sql] 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 @@ -177,7 +190,7 @@ private[sql] case class InMemoryRelation( .flatMap(_.values)) batchStats.add(stats) - CachedBatch(rowCount, columnBuilders.map { builder => + CachedBatchBytes(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) } @@ -185,32 +198,36 @@ private[sql] 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 = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, _cachedColumnBatches, batchStats) + _cachedColumnBuffers, batchStats) } override def newInstance(): this.type = { @@ -222,13 +239,19 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - _cachedColumnBatches, 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 27474dce673c..3952b14a331c 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 @@ private[sql] 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() } @@ -117,7 +120,7 @@ private[sql] 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") @@ -131,7 +134,7 @@ private[sql] 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.mapPartitionsInternal { cachedBatchIterator => val partitionFilter = newPredicate( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 73aaf89dab83..b15f38c2a71e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -323,7 +323,6 @@ abstract class QueryTest extends PlanTest { origin.child, l.tableName)( origin.cachedColumnBuffers, - origin.cachedColumnBatches, origin.batchStats) case p => p.transformExpressions { 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 120ad41e9325..d296b1ef2b8f 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 @@ -262,30 +262,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 0125aa2f24ee6ffc227a8df83917d25a2f9eb273 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 24 Jun 2016 14:53:10 -0700 Subject: [PATCH 20/20] 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 - */ } }