From 4bf995f35a4115302937eb04ccb99c90d9b3b681 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 9 Jul 2020 07:52:25 -0500 Subject: [PATCH 01/21] Make SQL cache serialization pluggable --- .../spark/sql/internal/StaticSQLConf.scala | 10 + .../spark/sql/execution/CacheManager.scala | 6 +- .../columnar/GenerateColumnAccessor.scala | 5 +- .../execution/columnar/InMemoryRelation.scala | 441 +++++++++++++++--- .../columnar/InMemoryTableScanExec.scala | 236 ++-------- .../columnar/InMemoryColumnarQuerySuite.scala | 43 +- 6 files changed, 464 insertions(+), 277 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 9618ff6062635..a612335f0be48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -126,6 +126,16 @@ object StaticSQLConf { .toSequence .createOptional + val SPARK_CACHE_SERIALIZER = buildStaticConf("spark.sql.cache.serializer") + .doc("The name of a class that implements " + + "org.apache.spark.sql.execution.columnar.CachedBatchSerializer. It will be used to " + + "translate SQL data into a format that can more efficiently be cached. The underlying " + + "API is subject to change so use with caution. Multiple classes cannot be specified. " + + "The class must have a no-arg constructor.") + .version("3.1.0") + .stringConf + .createWithDefault("org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer") + val QUERY_EXECUTION_LISTENERS = buildStaticConf("spark.sql.queryExecutionListeners") .doc("List of class names implementing QueryExecutionListener that will be automatically " + "added to newly created sessions. The classes should have either a no-arg constructor, " + 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 7d86c48015406..aaceba1165e70 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 @@ -27,11 +27,10 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.columnar.{DefaultCachedBatchSerializer, InMemoryRelation} import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -85,8 +84,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val inMemoryRelation = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(planToCache) InMemoryRelation( - sessionWithAqeOff.sessionState.conf.useCompression, - sessionWithAqeOff.sessionState.conf.columnBatchSize, storageLevel, + storageLevel, qe.executedPlan, tableName, optimizedPlan = qe.optimizedPlan) 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 bd2d06665a910..eb0663830dd67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.CalendarInterval * An Iterator to walk through the InternalRows from a CachedBatch */ abstract class ColumnarIterator extends Iterator[InternalRow] { - def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], + def initialize(input: Iterator[DefaultCachedBatch], columnTypes: Array[DataType], columnIndexes: Array[Int]): Unit } @@ -203,7 +203,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return false; } - ${classOf[CachedBatch].getName} batch = (${classOf[CachedBatch].getName}) input.next(); + ${classOf[DefaultCachedBatch].getName} batch = + (${classOf[DefaultCachedBatch].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 c8fa07941af87..a4fd8741460e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -19,76 +19,280 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging 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, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.LongAccumulator +import org.apache.spark.util.{LongAccumulator, Utils} +/** + * Basic interface that all cached batches of data must support. This is primarily to allow + * for metrics to be handled outside of the encoding and decoding steps in a standard way. + */ +trait CachedBatch { + def numRows: Int + def sizeInBytes: Long +} /** - * 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 + * Provides APIs for compressing, filtering, and decompressing SQL data that will be + * persisted/cached. */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +trait CachedBatchSerializer extends Serializable { + /** + * Run the given plan and convert its output to a implementation of [[CachedBatch]]. + * @param cachedPlan the plan to run. + * @return the RDD containing the batches of data to cache. + */ + def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] + + /** + * Builds a function that can be used to filter which batches are loaded. + * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide what + * you need with the added expense of calculating the min and max value for some + * data columns, depending on their data type. Note that this is intended to skip batches + * that are not needed, and the actual filtering of individual rows is handled later. + * @param predicates the set of expressions to use for filtering. + * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful + * if you don't store it with the data. + * @return a function that takes the partition id and the iterator of batches in the partition. + * It returns an iterator of batches that should be loaded. + */ + def buildFilter(predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] + + /** + * Decompress the cached data into a ColumnarBatch. This currently is only used for basic types + * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType + * That may change in the future. + * @param input the cached batches that should be decompressed. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data, and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return the batches in the ColumnarBatch format. + */ + def decompressColumnar(input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] + + /** + * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code + * generation is advised. + * @param input the cached batches that should be decompressed. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data, and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return the rows that were stored in the cached batches. + */ + def decompressToRows(input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] +} -case class CachedRDDBuilder( - useCompression: Boolean, - batchSize: Int, - storageLevel: StorageLevel, - @transient cachedPlan: SparkPlan, - tableName: Option[String]) { +/** + * A [[CachedBatch]] that stored some simple metrics that can be used for filtering of batches with + * the [[SimpleMetricsCachedBatchSerializer]]. + */ +trait SimpleMetricsCachedBatch extends CachedBatch { + /** + * Holds the same as ColumnStats. + * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long + */ + val stats: InternalRow + override def sizeInBytes: Long = stats.getLong(4) +} - @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null +// Currently, only use statistics from atomic types except binary type only. +private object ExtractableLiteral { + def unapply(expr: Expression): Option[Literal] = expr match { + case lit: Literal => lit.dataType match { + case BinaryType => None + case _: AtomicType => Some(lit) + case _ => None + } + case _ => None + } +} - val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator - val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator +/** + * Provides basic filtering for [[CachedBatchSerializer]] implementations. + */ +trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { + override def buildFilter(predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): + (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + val stats = new PartitionStatistics(cachedAttributes) + val statsSchema = stats.schema + + def statsFor(a: Attribute): ColumnStatisticsSchema = { + stats.forAttribute(a) + } - val cachedName = tableName.map(n => s"In-memory table $n") - .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024)) + // 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 lazy 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, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualTo(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l + case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound + + case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l + case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + l <= statsFor(a).upperBound + + case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound + case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + l <= statsFor(a).upperBound + case GreaterThanOrEqual(ExtractableLiteral(l), 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 + + case In(a: AttributeReference, list: Seq[Expression]) + if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => + list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && + l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) + // This is an example to explain how it works, imagine that the id column stored as follows: + // __________________________________________ + // | Partition ID | lowerBound | upperBound | + // |--------------|------------|------------| + // | p1 | '1' | '9' | + // | p2 | '10' | '19' | + // | p3 | '20' | '29' | + // | p4 | '30' | '39' | + // | p5 | '40' | '49' | + // |______________|____________|____________| + // + // A filter: df.filter($"id".startsWith("2")). + // In this case it substr lowerBound and upperBound: + // ________________________________________________________________________________________ + // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | + // |--------------|-----------------------------------|-----------------------------------| + // | p1 | '1' | '9' | + // | p2 | '1' | '1' | + // | p3 | '2' | '2' | + // | p4 | '3' | '3' | + // | p5 | '4' | '4' | + // |______________|___________________________________|___________________________________| + // + // We can see that we only need to read p1 and p3. + case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound.substr(0, Length(l)) <= l && + l <= statsFor(a).upperBound.substr(0, Length(l)) + } - def cachedColumnBuffers: RDD[CachedBatch] = { - if (_cachedColumnBuffers == null) { - synchronized { - if (_cachedColumnBuffers == null) { - _cachedColumnBuffers = buildBuffers() - } + // When we bind the filters we need to do it against the stats schema + val partitionFilters: Seq[Expression] = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + statsSchema, + 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) } } - _cachedColumnBuffers - } - def clearCache(blocking: Boolean = false): Unit = { - if (_cachedColumnBuffers != null) { - synchronized { - if (_cachedColumnBuffers != null) { - _cachedColumnBuffers.unpersist(blocking) - _cachedColumnBuffers = null + def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = { + val partitionFilter = Predicate.create( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + cachedAttributes) + + partitionFilter.initialize(index) + val schemaIndex = cachedAttributes.zipWithIndex + + cachedBatchIterator.filter { cb => + val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch] + if (!partitionFilter.eval(cachedBatch.stats)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true } } } + + ret } +} - def isCachedColumnBuffersLoaded: Boolean = { - _cachedColumnBuffers != null +/** + * The default implementation of CachedBatch. + * + * @param numRows The total number of rows in this batch + * @param buffers The buffers for serialized columns + * @param stats The stat of columns + */ +case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + extends SimpleMetricsCachedBatch + +/** + * The default implementation of CachedBatchSerializer. + */ +class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { + override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = { + val batchSize = cachedPlan.conf.columnBatchSize + val useCompression = cachedPlan.conf.useCompression + convertForCacheInternal(cachedPlan, batchSize, useCompression) } - private def buildBuffers(): RDD[CachedBatch] = { + def convertForCacheInternal(cachedPlan: SparkPlan, + batchSize: Int, + useCompression: Boolean): RDD[CachedBatch] = { val output = cachedPlan.output - val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { + cachedPlan.execute().mapPartitionsInternal { rowIterator => + new Iterator[DefaultCachedBatch] { + def next(): DefaultCachedBatch = { val columnBuilders = output.map { attribute => ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray @@ -96,7 +300,7 @@ case class CachedRDDBuilder( var rowCount = 0 var totalSize = 0L while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { val row = rowIterator.next() // Added for SPARK-6082. This assertion can be useful for scenarios when something @@ -106,8 +310,8 @@ case class CachedRDDBuilder( assert( row.numFields == columnBuilders.length, s"Row column number mismatch, expected ${output.size} columns, " + - s"but got ${row.numFields}." + - s"\nRow content: $row") + s"but got ${row.numFields}." + + s"\nRow content: $row") var i = 0 totalSize = 0 @@ -119,20 +323,125 @@ case class CachedRDDBuilder( rowCount += 1 } - sizeInBytesStats.add(totalSize) - rowCountStats.add(rowCount) - val stats = InternalRow.fromSeq( columnBuilders.flatMap(_.columnStats.collectedStatistics)) - CachedBatch(rowCount, columnBuilders.map { builder => + DefaultCachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) } def hasNext: Boolean = rowIterator.hasNext } - }.persist(storageLevel) + } + } + + override def decompressColumnar(input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + + val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled + val outputSchema = StructType.fromAttributes(selectedAttributes) + val columnIndices = + selectedAttributes.map(a => cacheAttributes.map(o => o.exprId).indexOf(a.exprId)).toArray + + def createAndDecompressColumn(cb: CachedBatch): ColumnarBatch = { + val cachedColumnarBatch = cb.asInstanceOf[DefaultCachedBatch] + val rowCount = cachedColumnarBatch.numRows + val taskContext = Option(TaskContext.get()) + val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { + OnHeapColumnVector.allocateColumns(rowCount, outputSchema) + } else { + OffHeapColumnVector.allocateColumns(rowCount, outputSchema) + } + val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) + columnarBatch.setNumRows(rowCount) + + for (i <- selectedAttributes.indices) { + ColumnAccessor.decompress( + cachedColumnarBatch.buffers(columnIndices(i)), + columnarBatch.column(i).asInstanceOf[WritableColumnVector], + outputSchema.fields(i).dataType, rowCount) + } + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) + columnarBatch + } + input.map(createAndDecompressColumn) + } + + override def decompressToRows(input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + selectedAttributes.map { a => + cacheAttributes.map(_.exprId).indexOf(a.exprId) -> a.dataType + }.unzip + + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + + input.mapPartitionsInternal { cachedBatchIterator => + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(cachedBatchIterator.asInstanceOf[Iterator[DefaultCachedBatch]], + columnTypes, + requestedColumnIndices.toArray) + columnarIterator + } + } +} + +case class CachedRDDBuilder( + serializer: CachedBatchSerializer, + storageLevel: StorageLevel, + @transient cachedPlan: SparkPlan, + tableName: Option[String]) { + + @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null + + val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator + val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator + + val cachedName = tableName.map(n => s"In-memory table $n") + .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024)) + + def cachedColumnBuffers: RDD[CachedBatch] = { + if (_cachedColumnBuffers == null) { + synchronized { + if (_cachedColumnBuffers == null) { + _cachedColumnBuffers = buildBuffers() + } + } + } + _cachedColumnBuffers + } + + def clearCache(blocking: Boolean = false): Unit = { + if (_cachedColumnBuffers != null) { + synchronized { + if (_cachedColumnBuffers != null) { + _cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null + } + } + } + } + + def isCachedColumnBuffersLoaded: Boolean = { + _cachedColumnBuffers != null + } + + private def buildBuffers(): RDD[CachedBatch] = { + val cached = serializer.convertForCache(cachedPlan) + .map { batch => + sizeInBytesStats.add(batch.sizeInBytes) + rowCountStats.add(batch.numRows) + batch + }.persist(storageLevel) cached.setName(cachedName) cached } @@ -140,14 +449,42 @@ case class CachedRDDBuilder( object InMemoryRelation { + private[this] var ser: Option[CachedBatchSerializer] = None + private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = { + if (ser.isEmpty) { + synchronized { + if (ser.isEmpty) { + val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) + val serClass = Utils.classForName(serName) + val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer] + ser = Some(instance) + } + } + } + ser.get + } + def apply( - useCompression: Boolean, - batchSize: Int, storageLevel: StorageLevel, child: SparkPlan, tableName: Option[String], optimizedPlan: LogicalPlan): InMemoryRelation = { - val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName) + val cacheBuilder = CachedRDDBuilder(getSerializer(child.conf), storageLevel, child, tableName) + val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) + relation.statsOfPlanToCache = optimizedPlan.stats + relation + } + + /** + * This API is intended only to be used for testing. + */ + def apply( + serializer: CachedBatchSerializer, + storageLevel: StorageLevel, + child: SparkPlan, + tableName: Option[String], + optimizedPlan: LogicalPlan): InMemoryRelation = { + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation 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 f03c2586048bd..37c18a4326b12 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,10 +17,8 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.TaskContext 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.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning @@ -28,7 +26,7 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodege import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.ColumnarBatch case class InMemoryTableScanExec( @@ -79,46 +77,14 @@ case class InMemoryTableScanExec( }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) } - private val columnIndices = - attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray - - private val relationSchema = relation.schema.toArray - - private lazy val columnarBatchSchema = new StructType(columnIndices.map(i => relationSchema(i))) - - private def createAndDecompressColumn( - cachedColumnarBatch: CachedBatch, - offHeapColumnVectorEnabled: Boolean): ColumnarBatch = { - val rowCount = cachedColumnarBatch.numRows - val taskContext = Option(TaskContext.get()) - val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { - OnHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } else { - OffHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } - val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) - columnarBatch.setNumRows(rowCount) - - for (i <- attributes.indices) { - ColumnAccessor.decompress( - cachedColumnarBatch.buffers(columnIndices(i)), - columnarBatch.column(i).asInstanceOf[WritableColumnVector], - columnarBatchSchema.fields(i).dataType, rowCount) - } - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) - columnarBatch - } - private lazy val columnarInputRDD: RDD[ColumnarBatch] = { val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() - val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled - buffers - .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) - .map { buffer => - numOutputRows += buffer.numRows() - buffer - } + relation.cacheBuilder.serializer.decompressColumnar(buffers, relation.output, attributes, conf) + .map { cb => + numOutputRows += cb.numRows() + cb + } } private lazy val inputRDD: RDD[InternalRow] = { @@ -130,34 +96,34 @@ case class InMemoryTableScanExec( val numOutputRows = longMetric("numOutputRows") // Using these variables here to avoid serialization of entire objects (if referenced // directly) within the map Partitions closure. - val relOutput: AttributeSeq = relation.output - - filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip + val relOutput = relation.output + val serializer = relation.cacheBuilder.serializer - // update SQL metrics - val withMetrics = cachedBatchIterator.map { batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) - } - numOutputRows += batch.numRows + // update SQL metrics + val withMetrics = if (enableAccumulatorsForTest) { + filteredCachedBatches().map{ batch => + readBatches.add(1) 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 (enableAccumulatorsForTest && columnarIterator.hasNext) { - readPartitions.add(1) + } else { + filteredCachedBatches() + }.map{ batch => + numOutputRows += batch.numRows + batch + } + val ret = serializer.decompressToRows(withMetrics, relOutput, attributes, conf) + if (enableAccumulatorsForTest) { + @transient var seen = false + def incParts(index: Int, iter: Iterator[InternalRow]): Iterator[InternalRow] = { + if (!seen) { + readPartitions.add(1) + } + seen = true + iter } - columnarIterator + ret.mapPartitionsWithIndexInternal(incParts) + } else { + ret } } @@ -186,114 +152,6 @@ case class InMemoryTableScanExec( override def outputOrdering: Seq[SortOrder] = relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder]) - // Keeps relation's partition statistics because we don't serialize relation. - private val stats = relation.partitionStatistics - private def statsFor(a: Attribute) = stats.forAttribute(a) - - // Currently, only use statistics from atomic types except binary type only. - private object ExtractableLiteral { - def unapply(expr: Expression): Option[Literal] = expr match { - case lit: Literal => lit.dataType match { - case BinaryType => None - case _: AtomicType => Some(lit) - case _ => None - } - case _ => None - } - } - - // 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 lazy 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, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualTo(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l - case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound - - case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l - case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - l <= statsFor(a).upperBound - - case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound - case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l - - case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - l <= statsFor(a).upperBound - case GreaterThanOrEqual(ExtractableLiteral(l), 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 - - case In(a: AttributeReference, list: Seq[Expression]) - if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => - list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && - l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) - - // This is an example to explain how it works, imagine that the id column stored as follows: - // __________________________________________ - // | Partition ID | lowerBound | upperBound | - // |--------------|------------|------------| - // | p1 | '1' | '9' | - // | p2 | '10' | '19' | - // | p3 | '20' | '29' | - // | p4 | '30' | '39' | - // | p5 | '40' | '49' | - // |______________|____________|____________| - // - // A filter: df.filter($"id".startsWith("2")). - // In this case it substr lowerBound and upperBound: - // ________________________________________________________________________________________ - // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | - // |--------------|-----------------------------------|-----------------------------------| - // | p1 | '1' | '9' | - // | p2 | '1' | '1' | - // | p3 | '2' | '2' | - // | p4 | '3' | '3' | - // | p5 | '4' | '4' | - // |______________|___________________________________|___________________________________| - // - // We can see that we only need to read p1 and p3. - case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound.substr(0, Length(l)) <= l && - l <= statsFor(a).upperBound.substr(0, Length(l)) - } - - lazy val partitionFilters: Seq[Expression] = { - predicates.flatMap { p => - val filter = buildFilter.lift(p) - val boundFilter = - filter.map( - BindReferences.bindReference( - _, - stats.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 enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes @@ -303,37 +161,13 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning private def filteredCachedBatches(): RDD[CachedBatch] = { - // Using these variables here to avoid serialization of entire objects (if referenced directly) - // within the map Partitions closure. - val schema = stats.schema - val schemaIndex = schema.zipWithIndex val buffers = relation.cacheBuilder.cachedColumnBuffers - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = Predicate.create( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - // Do partition batch pruning if enabled - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter.eval(cachedBatch.stats)) { - logDebug { - val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - s"Skipping partition based on stats $statsString" - } - false - } else { - true - } - } - } else { - cachedBatchIterator - } + if (inMemoryPartitionPruningEnabled) { + val filterFunc = relation.cacheBuilder.serializer.buildFilter(predicates, relation.output) + buffers.mapPartitionsWithIndexInternal(filterFunc) + } else { + buffers } } 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 18f29f7b90ad5..ae3bebb13f4ab 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,18 +20,26 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{LongAccumulator, Utils} + +class TestCachedBatchSerializer( + useCompression: Boolean, + batchSize: Int) extends DefaultCachedBatchSerializer { + override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = + convertForCacheInternal(cachedPlan, batchSize, useCompression) +} class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -42,12 +50,13 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { data.createOrReplaceTempView(s"testData$dataType") val storageLevel = MEMORY_ONLY val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None, + val inMemoryRelation = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + storageLevel, plan, None, data.logicalPlan) assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) inMemoryRelation.cacheBuilder.cachedColumnBuffers.collect().head match { - case _: CachedBatch => + case _: DefaultCachedBatch => case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") } checkAnswer(inMemoryRelation, data.collect().toSeq) @@ -119,8 +128,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("simple columnar query") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) } @@ -140,8 +149,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("projection") { val logicalPlan = testData.select('value, 'key).logicalPlan val plan = spark.sessionState.executePlan(logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, logicalPlan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -157,8 +166,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) @@ -336,7 +345,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-17549: cached table size should be correctly calculated") { val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -349,7 +359,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("cached row count should be calculated") { val data = spark.range(6).toDF val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -474,12 +485,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") { val attribute = AttributeReference("a", IntegerType)() - val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil) - val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None, - LocalRelation(Seq(attribute), Nil)) - val tableScanExec = InMemoryTableScanExec(Seq(attribute), - Seq(In(attribute, Nil)), testRelation) - assert(tableScanExec.partitionFilters.isEmpty) + val testSerializer = new TestCachedBatchSerializer(false, 1) + testSerializer.buildFilter(Seq(In(attribute, Nil)), Seq(attribute)) } testWithWholeStageCodegenOnAndOff("SPARK-22348: table cache " + From 427fc68fce9e030aae160e123df25e696aea1e5c Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 10 Jul 2020 16:25:34 -0500 Subject: [PATCH 02/21] Addressed review comments --- .../execution/columnar/InMemoryRelation.scala | 27 +++++++++++++------ .../columnar/InMemoryTableScanExec.scala | 8 +++--- .../columnar/InMemoryColumnarQuerySuite.scala | 5 ++-- 3 files changed, 25 insertions(+), 15 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 a4fd8741460e7..a463eb5db78e9 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils import org.apache.spark.TaskContext +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD @@ -42,6 +43,8 @@ import org.apache.spark.util.{LongAccumulator, Utils} * Basic interface that all cached batches of data must support. This is primarily to allow * for metrics to be handled outside of the encoding and decoding steps in a standard way. */ +@DeveloperApi +@Since("3.1.0") trait CachedBatch { def numRows: Int def sizeInBytes: Long @@ -51,6 +54,8 @@ trait CachedBatch { * Provides APIs for compressing, filtering, and decompressing SQL data that will be * persisted/cached. */ +@DeveloperApi +@Since("3.1.0") trait CachedBatchSerializer extends Serializable { /** * Run the given plan and convert its output to a implementation of [[CachedBatch]]. @@ -83,9 +88,10 @@ trait CachedBatchSerializer extends Serializable { * @param selectedAttributes the field that should be loaded from the data, and the order they * should appear in the output batch. * @param conf the configuration for the job. - * @return the batches in the ColumnarBatch format. + * @return an RDD of the input cached batches transformed into the ColumnarBatch format. */ - def decompressColumnar(input: RDD[CachedBatch], + def decompressColumnar( + input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] @@ -100,16 +106,19 @@ trait CachedBatchSerializer extends Serializable { * @param conf the configuration for the job. * @return the rows that were stored in the cached batches. */ - def decompressToRows(input: RDD[CachedBatch], + def decompressToRows( + input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[InternalRow] } /** - * A [[CachedBatch]] that stored some simple metrics that can be used for filtering of batches with + * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with * the [[SimpleMetricsCachedBatchSerializer]]. */ +@DeveloperApi +@Since("3.1.0") trait SimpleMetricsCachedBatch extends CachedBatch { /** * Holds the same as ColumnStats. @@ -134,10 +143,11 @@ private object ExtractableLiteral { /** * Provides basic filtering for [[CachedBatchSerializer]] implementations. */ +@DeveloperApi +@Since("3.1.0") trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { override def buildFilter(predicates: Seq[Expression], - cachedAttributes: Seq[Attribute]): - (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { val stats = new PartitionStatistics(cachedAttributes) val statsSchema = stats.schema @@ -261,7 +271,6 @@ trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logg } } } - ret } } @@ -273,12 +282,14 @@ trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logg * @param buffers The buffers for serialized columns * @param stats The stat of columns */ +private[sql] case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) extends SimpleMetricsCachedBatch /** * The default implementation of CachedBatchSerializer. */ +private[sql] class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = { val batchSize = cachedPlan.conf.columnBatchSize @@ -339,7 +350,6 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] = { - val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled val outputSchema = StructType.fromAttributes(selectedAttributes) val columnIndices = @@ -395,6 +405,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } } +private[sql] case class CachedRDDBuilder( serializer: CachedBatchSerializer, storageLevel: StorageLevel, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 37c18a4326b12..39b3bee0d9019 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 @@ -81,10 +81,10 @@ case class InMemoryTableScanExec( val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() relation.cacheBuilder.serializer.decompressColumnar(buffers, relation.output, attributes, conf) - .map { cb => - numOutputRows += cb.numRows() - cb - } + .map { cb => + numOutputRows += cb.numRows() + cb + } } private lazy val inputRDD: RDD[InternalRow] = { 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 ae3bebb13f4ab..9b9c63e7b873e 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 @@ -23,16 +23,15 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.{LongAccumulator, Utils} + class TestCachedBatchSerializer( useCompression: Boolean, From 0361237986e10eec828cacb400ee6c948fee30fd Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 10 Jul 2020 16:28:24 -0500 Subject: [PATCH 03/21] Few more --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 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 a463eb5db78e9..0c2bd1a13ab0a 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 @@ -97,14 +97,14 @@ trait CachedBatchSerializer extends Serializable { conf: SQLConf): RDD[ColumnarBatch] /** - * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code + * Decompress the cached batch into [[InternalRow]]. If you want this to be performant, code * generation is advised. * @param input the cached batches that should be decompressed. * @param cacheAttributes the attributes of the data in the batch. * @param selectedAttributes the field that should be loaded from the data, and the order they * should appear in the output batch. * @param conf the configuration for the job. - * @return the rows that were stored in the cached batches. + * @return RDD of the rows that were stored in the cached batches. */ def decompressToRows( input: RDD[CachedBatch], @@ -346,7 +346,8 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } } - override def decompressColumnar(input: RDD[CachedBatch], + override def decompressColumnar( + input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] = { From f9895727afdbff2af1254945bf0b345a7835df9c Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 14 Jul 2020 15:48:20 -0500 Subject: [PATCH 04/21] Fixed one set of tests --- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 4 +--- 1 file changed, 1 insertion(+), 3 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 39b3bee0d9019..e59ae4b5f7990 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 @@ -113,12 +113,10 @@ case class InMemoryTableScanExec( } val ret = serializer.decompressToRows(withMetrics, relOutput, attributes, conf) if (enableAccumulatorsForTest) { - @transient var seen = false def incParts(index: Int, iter: Iterator[InternalRow]): Iterator[InternalRow] = { - if (!seen) { + if (iter.hasNext) { readPartitions.add(1) } - seen = true iter } ret.mapPartitionsWithIndexInternal(incParts) From ad11a794a634488f4b01b7e735179b74c2619774 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 14 Jul 2020 16:26:15 -0500 Subject: [PATCH 05/21] Fixed the other bug --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 4 +++- 1 file changed, 3 insertions(+), 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 efe36fe2fa431..2ad147df4a05a 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 @@ -123,9 +123,11 @@ trait SimpleMetricsCachedBatch extends CachedBatch { /** * Holds the same as ColumnStats. * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long + * Which is repeated for each column in the original data. */ val stats: InternalRow - override def sizeInBytes: Long = stats.getLong(4) + override def sizeInBytes: Long = + Range.apply(4, stats.numFields, 5).map(stats.getLong).sum } // Currently, only use statistics from atomic types except binary type only. From cb29e474fb7ad007734fd0a91a6329e7508e627a Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 17 Jul 2020 11:15:09 -0500 Subject: [PATCH 06/21] Addressed review comments --- .../spark/sql/internal/StaticSQLConf.scala | 2 +- .../sql/columnar/CachedBatchSerializer.scala | 279 ++++++++++++++++++ .../sql/execution/columnar/ColumnStats.scala | 4 +- .../execution/columnar/InMemoryRelation.scala | 246 +-------------- .../columnar/InMemoryTableScanExec.scala | 1 + .../columnar/InMemoryColumnarQuerySuite.scala | 2 +- 6 files changed, 286 insertions(+), 248 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index a612335f0be48..adbbc6e540d17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -128,7 +128,7 @@ object StaticSQLConf { val SPARK_CACHE_SERIALIZER = buildStaticConf("spark.sql.cache.serializer") .doc("The name of a class that implements " + - "org.apache.spark.sql.execution.columnar.CachedBatchSerializer. It will be used to " + + "org.apache.spark.sql.columnar.CachedBatchSerializer. It will be used to " + "translate SQL data into a format that can more efficiently be cached. The underlying " + "API is subject to change so use with caution. Multiple classes cannot be specified. " + "The class must have a no-arg constructor.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala new file mode 100644 index 0000000000000..27ab587671682 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +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.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{AtomicType, BinaryType} +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * Basic interface that all cached batches of data must support. This is primarily to allow + * for metrics to be handled outside of the encoding and decoding steps in a standard way. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatch { + def numRows: Int + def sizeInBytes: Long +} + +/** + * Provides APIs for compressing, filtering, and decompressing SQL data that will be + * persisted/cached. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatchSerializer extends Serializable { + /** + * Run the given plan and convert its output to a implementation of [[CachedBatch]]. + * @param cachedPlan the plan to run. + * @return the RDD containing the batches of data to cache. + */ + def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] + + /** + * Builds a function that can be used to filter which batches are loaded. + * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide the filter logic + * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] + * provides the APIs to hold those metrics and explains the metrics used, really just min and max. + * Note that this is intended to skip batches that are not needed, and the actual filtering of + * individual rows is handled later. + * @param predicates the set of expressions to use for filtering. + * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful + * if you don't store it with the data. + * @return a function that takes the partition id and the iterator of batches in the partition. + * It returns an iterator of batches that should be loaded. + */ + def buildFilter(predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] + + /** + * Decompress the cached data into a ColumnarBatch. This currently is only used for basic types + * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType + * That may change in the future. + * @param input the cached batches that should be decompressed. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data, and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return an RDD of the input cached batches transformed into the ColumnarBatch format. + */ + def decompressColumnar( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] + + /** + * Decompress the cached batch into [[InternalRow]]. If you want this to be performant, code + * generation is advised. + * @param input the cached batches that should be decompressed. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data, and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return RDD of the rows that were stored in the cached batches. + */ + def decompressToRows( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] +} + +/** + * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with + * the [[SimpleMetricsCachedBatchSerializer]]. + * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata + * are needed in the row. + */ +@DeveloperApi +@Since("3.1.0") +trait SimpleMetricsCachedBatch extends CachedBatch { + /** + * Holds the same as ColumnStats. + * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long + * Which is repeated for each column in the original data. + */ + val stats: InternalRow + override def sizeInBytes: Long = + Range.apply(4, stats.numFields, 5).map(stats.getLong).sum +} + +// Currently, only use statistics from atomic types except binary type only. +private object ExtractableLiteral { + def unapply(expr: Expression): Option[Literal] = expr match { + case lit: Literal => lit.dataType match { + case BinaryType => None + case _: AtomicType => Some(lit) + case _ => None + } + case _ => None + } +} + +/** + * Provides basic filtering for [[CachedBatchSerializer]] implementations. + * The requirement to extend this is that all of the batches produced by your serializer are + * instances of [[SimpleMetricsCachedBatch]]. + * This does not calculate the metrics needed to be stored in the batches. That is up to each + * implementation. The metrics required are really just min and max values and those are optional + * especially for complex types. Because those metrics are simple and it is likely that compression + * will also be done on the data we thought it best to let each implementation decide on the most + * efficient way to calculate the metrics, possibly combining them with compression passes that + * might also be done across the data. + */ +@DeveloperApi +@Since("3.1.0") +abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { + override def buildFilter(predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + val stats = new PartitionStatistics(cachedAttributes) + val statsSchema = stats.schema + + def statsFor(a: Attribute): ColumnStatisticsSchema = { + stats.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 lazy 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, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualTo(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l + case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound + + case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l + case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + l <= statsFor(a).upperBound + + case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound + case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + l <= statsFor(a).upperBound + case GreaterThanOrEqual(ExtractableLiteral(l), 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 + + case In(a: AttributeReference, list: Seq[Expression]) + if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => + list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && + l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) + // This is an example to explain how it works, imagine that the id column stored as follows: + // __________________________________________ + // | Partition ID | lowerBound | upperBound | + // |--------------|------------|------------| + // | p1 | '1' | '9' | + // | p2 | '10' | '19' | + // | p3 | '20' | '29' | + // | p4 | '30' | '39' | + // | p5 | '40' | '49' | + // |______________|____________|____________| + // + // A filter: df.filter($"id".startsWith("2")). + // In this case it substr lowerBound and upperBound: + // ________________________________________________________________________________________ + // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | + // |--------------|-----------------------------------|-----------------------------------| + // | p1 | '1' | '9' | + // | p2 | '1' | '1' | + // | p3 | '2' | '2' | + // | p4 | '3' | '3' | + // | p5 | '4' | '4' | + // |______________|___________________________________|___________________________________| + // + // We can see that we only need to read p1 and p3. + case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound.substr(0, Length(l)) <= l && + l <= statsFor(a).upperBound.substr(0, Length(l)) + } + + // When we bind the filters we need to do it against the stats schema + val partitionFilters: Seq[Expression] = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + statsSchema, + 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) + } + } + + def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = { + val partitionFilter = Predicate.create( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + cachedAttributes) + + partitionFilter.initialize(index) + val schemaIndex = cachedAttributes.zipWithIndex + + cachedBatchIterator.filter { cb => + val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch] + if (!partitionFilter.eval(cachedBatch.stats)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true + } + } + } + ret + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index 20ecc57c49e75..ba7bf9119cbbf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializable { +private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() @@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } -private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { +private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) (AttributeMap(allStats), allStats.flatMap(_._2.schema)) 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 2ad147df4a05a..bb9a779a69d4e 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 @@ -20,263 +20,23 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils import org.apache.spark.TaskContext -import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.internal.Logging 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.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType} +import org.apache.spark.sql.types.{StructType, UserDefinedType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{LongAccumulator, Utils} -/** - * Basic interface that all cached batches of data must support. This is primarily to allow - * for metrics to be handled outside of the encoding and decoding steps in a standard way. - */ -@DeveloperApi -@Since("3.1.0") -trait CachedBatch { - def numRows: Int - def sizeInBytes: Long -} - -/** - * Provides APIs for compressing, filtering, and decompressing SQL data that will be - * persisted/cached. - */ -@DeveloperApi -@Since("3.1.0") -trait CachedBatchSerializer extends Serializable { - /** - * Run the given plan and convert its output to a implementation of [[CachedBatch]]. - * @param cachedPlan the plan to run. - * @return the RDD containing the batches of data to cache. - */ - def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] - - /** - * Builds a function that can be used to filter which batches are loaded. - * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide what - * you need with the added expense of calculating the min and max value for some - * data columns, depending on their data type. Note that this is intended to skip batches - * that are not needed, and the actual filtering of individual rows is handled later. - * @param predicates the set of expressions to use for filtering. - * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful - * if you don't store it with the data. - * @return a function that takes the partition id and the iterator of batches in the partition. - * It returns an iterator of batches that should be loaded. - */ - def buildFilter(predicates: Seq[Expression], - cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] - - /** - * Decompress the cached data into a ColumnarBatch. This currently is only used for basic types - * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType - * That may change in the future. - * @param input the cached batches that should be decompressed. - * @param cacheAttributes the attributes of the data in the batch. - * @param selectedAttributes the field that should be loaded from the data, and the order they - * should appear in the output batch. - * @param conf the configuration for the job. - * @return an RDD of the input cached batches transformed into the ColumnarBatch format. - */ - def decompressColumnar( - input: RDD[CachedBatch], - cacheAttributes: Seq[Attribute], - selectedAttributes: Seq[Attribute], - conf: SQLConf): RDD[ColumnarBatch] - - /** - * Decompress the cached batch into [[InternalRow]]. If you want this to be performant, code - * generation is advised. - * @param input the cached batches that should be decompressed. - * @param cacheAttributes the attributes of the data in the batch. - * @param selectedAttributes the field that should be loaded from the data, and the order they - * should appear in the output batch. - * @param conf the configuration for the job. - * @return RDD of the rows that were stored in the cached batches. - */ - def decompressToRows( - input: RDD[CachedBatch], - cacheAttributes: Seq[Attribute], - selectedAttributes: Seq[Attribute], - conf: SQLConf): RDD[InternalRow] -} - -/** - * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with - * the [[SimpleMetricsCachedBatchSerializer]]. - */ -@DeveloperApi -@Since("3.1.0") -trait SimpleMetricsCachedBatch extends CachedBatch { - /** - * Holds the same as ColumnStats. - * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long - * Which is repeated for each column in the original data. - */ - val stats: InternalRow - override def sizeInBytes: Long = - Range.apply(4, stats.numFields, 5).map(stats.getLong).sum -} - -// Currently, only use statistics from atomic types except binary type only. -private object ExtractableLiteral { - def unapply(expr: Expression): Option[Literal] = expr match { - case lit: Literal => lit.dataType match { - case BinaryType => None - case _: AtomicType => Some(lit) - case _ => None - } - case _ => None - } -} - -/** - * Provides basic filtering for [[CachedBatchSerializer]] implementations. - */ -@DeveloperApi -@Since("3.1.0") -trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { - override def buildFilter(predicates: Seq[Expression], - cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { - val stats = new PartitionStatistics(cachedAttributes) - val statsSchema = stats.schema - - def statsFor(a: Attribute): ColumnStatisticsSchema = { - stats.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 lazy 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, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualTo(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l - case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound - - case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l - case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - l <= statsFor(a).upperBound - - case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound - case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l - - case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - l <= statsFor(a).upperBound - case GreaterThanOrEqual(ExtractableLiteral(l), 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 - - case In(a: AttributeReference, list: Seq[Expression]) - if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => - list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && - l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) - // This is an example to explain how it works, imagine that the id column stored as follows: - // __________________________________________ - // | Partition ID | lowerBound | upperBound | - // |--------------|------------|------------| - // | p1 | '1' | '9' | - // | p2 | '10' | '19' | - // | p3 | '20' | '29' | - // | p4 | '30' | '39' | - // | p5 | '40' | '49' | - // |______________|____________|____________| - // - // A filter: df.filter($"id".startsWith("2")). - // In this case it substr lowerBound and upperBound: - // ________________________________________________________________________________________ - // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | - // |--------------|-----------------------------------|-----------------------------------| - // | p1 | '1' | '9' | - // | p2 | '1' | '1' | - // | p3 | '2' | '2' | - // | p4 | '3' | '3' | - // | p5 | '4' | '4' | - // |______________|___________________________________|___________________________________| - // - // We can see that we only need to read p1 and p3. - case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound.substr(0, Length(l)) <= l && - l <= statsFor(a).upperBound.substr(0, Length(l)) - } - - // When we bind the filters we need to do it against the stats schema - val partitionFilters: Seq[Expression] = { - predicates.flatMap { p => - val filter = buildFilter.lift(p) - val boundFilter = - filter.map( - BindReferences.bindReference( - _, - statsSchema, - 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) - } - } - - def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = { - val partitionFilter = Predicate.create( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - cachedAttributes) - - partitionFilter.initialize(index) - val schemaIndex = cachedAttributes.zipWithIndex - - cachedBatchIterator.filter { cb => - val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch] - if (!partitionFilter.eval(cachedBatch.stats)) { - logDebug { - val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - s"Skipping partition based on stats $statsString" - } - false - } else { - true - } - } - } - ret - } -} - /** * The default implementation of CachedBatch. * @@ -284,14 +44,12 @@ trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logg * @param buffers The buffers for serialized columns * @param stats The stat of columns */ -private[sql] case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) extends SimpleMetricsCachedBatch /** * The default implementation of CachedBatchSerializer. */ -private[sql] class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = { val batchSize = cachedPlan.conf.columnBatchSize 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 e59ae4b5f7990..3cdf4b7f160e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow 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.columnar.CachedBatch import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized._ 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 9b9c63e7b873e..efb684608e559 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 @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -32,7 +33,6 @@ import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel._ - class TestCachedBatchSerializer( useCompression: Boolean, batchSize: Int) extends DefaultCachedBatchSerializer { From f0c7cfe82cd91db8611ea8abc9b91e4df65a35f7 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 20 Jul 2020 10:04:58 -0500 Subject: [PATCH 07/21] Addressed review comments --- .../sql/columnar/CachedBatchSerializer.scala | 30 +++++++++++++------ .../execution/columnar/InMemoryRelation.scala | 25 +++++++++------- .../columnar/InMemoryTableScanExec.scala | 17 ++++------- .../columnar/InMemoryColumnarQuerySuite.scala | 3 +- 4 files changed, 43 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 27ab587671682..b65cac0a31c82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeRefer import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{AtomicType, BinaryType} +import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -55,9 +55,9 @@ trait CachedBatchSerializer extends Serializable { def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] /** - * Builds a function that can be used to filter which batches are loaded. + * Builds a function that can be used to filter batches prior to being decompressed. * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide the filter logic - * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] + * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] * provides the APIs to hold those metrics and explains the metrics used, really just min and max. * Note that this is intended to skip batches that are not needed, and the actual filtering of * individual rows is handled later. @@ -65,18 +65,30 @@ trait CachedBatchSerializer extends Serializable { * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful * if you don't store it with the data. * @return a function that takes the partition id and the iterator of batches in the partition. - * It returns an iterator of batches that should be loaded. + * It returns an iterator of batches that should be decompressed. */ def buildFilter(predicates: Seq[Expression], cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] /** - * Decompress the cached data into a ColumnarBatch. This currently is only used for basic types - * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType - * That may change in the future. + * Can [[decompressColumnar()]] be called instead of [[decompressToRows()]] for this given + * schema? True if it can and false if it cannot. Columnar output is typically preferred + * because it is more efficient. Note that [[decompressToRows()]] must always be supported + * as there are other checks that can force row based output. + * @param schema the schema of the data being checked. + * @return true if columnar output should be used for this schema, else false. + */ + def supportsColumnar(schema: StructType): Boolean + + /** + * Decompress the cached data into a ColumnarBatch. This currently is only used if + * [[supportsColumnar()]] returned true for the associated schema, but there are other checks + * that can force row based output. One of the main advantages of doing columnar output over row + * based output is the code generation is more standard and can be combined with code generation + * for downstream operations. * @param input the cached batches that should be decompressed. * @param cacheAttributes the attributes of the data in the batch. - * @param selectedAttributes the field that should be loaded from the data, and the order they + * @param selectedAttributes the fields that should be loaded from the data, and the order they * should appear in the output batch. * @param conf the configuration for the job. * @return an RDD of the input cached batches transformed into the ColumnarBatch format. @@ -88,7 +100,7 @@ trait CachedBatchSerializer extends Serializable { conf: SQLConf): RDD[ColumnarBatch] /** - * Decompress the cached batch into [[InternalRow]]. If you want this to be performant, code + * Decompress the cached batch into `InternalRow`. If you want this to be performant, code * generation is advised. * @param input the cached batches that should be decompressed. * @param cacheAttributes the attributes of the data in the batch. 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 bb9a779a69d4e..8934bef42a8c2 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 @@ -32,7 +32,7 @@ import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, Simple import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.sql.types.{StructType, UserDefinedType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{LongAccumulator, Utils} @@ -106,6 +106,15 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } } + override def supportsColumnar(schema: StructType): Boolean = schema.fields.forall(f => + f.dataType match { + // More types can be supported, but this is to match the original implementation that + // only supported primitive types "for ease of review" + case BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType => true + case _ => false + }) + override def decompressColumnar( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], @@ -222,16 +231,12 @@ case class CachedRDDBuilder( object InMemoryRelation { private[this] var ser: Option[CachedBatchSerializer] = None - private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = { + private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized { if (ser.isEmpty) { - synchronized { - if (ser.isEmpty) { - val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) - val serClass = Utils.classForName(serName) - val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer] - ser = Some(instance) - } - } + val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) + val serClass = Utils.classForName(serName) + val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer] + ser = Some(instance) } ser.get } 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 3cdf4b7f160e2..a912c4f961a3b 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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized._ -import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -69,13 +68,9 @@ case class InMemoryTableScanExec( * If false, get data from UnsafeRow build from CachedBatch */ override val supportsColumnar: Boolean = { - // In the initial implementation, for ease of review - // support only primitive data types and # of fields is less than wholeStageMaxNumFields - conf.cacheVectorizedReaderEnabled && relation.schema.fields.forall(f => f.dataType match { - case BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType => true - case _ => false - }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) + conf.cacheVectorizedReaderEnabled && + !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) && + relation.cacheBuilder.serializer.supportsColumnar(relation.schema) } private lazy val columnarInputRDD: RDD[ColumnarBatch] = { @@ -112,7 +107,7 @@ case class InMemoryTableScanExec( numOutputRows += batch.numRows batch } - val ret = serializer.decompressToRows(withMetrics, relOutput, attributes, conf) + val decompressedRows = serializer.decompressToRows(withMetrics, relOutput, attributes, conf) if (enableAccumulatorsForTest) { def incParts(index: Int, iter: Iterator[InternalRow]): Iterator[InternalRow] = { if (iter.hasNext) { @@ -120,9 +115,9 @@ case class InMemoryTableScanExec( } iter } - ret.mapPartitionsWithIndexInternal(incParts) + decompressedRows.mapPartitionsWithIndexInternal(incParts) } else { - ret + decompressedRows } } 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 efb684608e559..8897c150eac62 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 @@ -50,8 +50,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { val storageLevel = MEMORY_ONLY val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan val inMemoryRelation = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), - storageLevel, plan, None, - data.logicalPlan) + storageLevel, plan, None, data.logicalPlan) assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) inMemoryRelation.cacheBuilder.cachedColumnBuffers.collect().head match { From 46f40216fd246f4e81cbaa6ce930287bf4c92975 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 20 Jul 2020 10:42:17 -0500 Subject: [PATCH 08/21] Fixed some docs issues --- .../apache/spark/sql/columnar/CachedBatchSerializer.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index b65cac0a31c82..a92038635450d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -71,9 +71,9 @@ trait CachedBatchSerializer extends Serializable { cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] /** - * Can [[decompressColumnar()]] be called instead of [[decompressToRows()]] for this given + * Can `decompressColumnar()` be called instead of `decompressToRows()` for this given * schema? True if it can and false if it cannot. Columnar output is typically preferred - * because it is more efficient. Note that [[decompressToRows()]] must always be supported + * because it is more efficient. Note that `decompressToRows()` must always be supported * as there are other checks that can force row based output. * @param schema the schema of the data being checked. * @return true if columnar output should be used for this schema, else false. @@ -82,7 +82,7 @@ trait CachedBatchSerializer extends Serializable { /** * Decompress the cached data into a ColumnarBatch. This currently is only used if - * [[supportsColumnar()]] returned true for the associated schema, but there are other checks + * `supportsColumnar()`` returned true for the associated schema, but there are other checks * that can force row based output. One of the main advantages of doing columnar output over row * based output is the code generation is more standard and can be combined with code generation * for downstream operations. From 10b0ce193d2f711ddc5702465c847d0a94cde313 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 20 Jul 2020 11:28:04 -0500 Subject: [PATCH 09/21] Fixed bad comment quoting, and added comments for reviews --- .../org/apache/spark/sql/columnar/CachedBatchSerializer.scala | 4 +++- .../spark/sql/execution/columnar/InMemoryRelation.scala | 3 +++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index a92038635450d..b5cd49c2ab0a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -82,7 +82,7 @@ trait CachedBatchSerializer extends Serializable { /** * Decompress the cached data into a ColumnarBatch. This currently is only used if - * `supportsColumnar()`` returned true for the associated schema, but there are other checks + * `supportsColumnar()` returned true for the associated schema, but there are other checks * that can force row based output. One of the main advantages of doing columnar output over row * based output is the code generation is more standard and can be combined with code generation * for downstream operations. @@ -163,6 +163,8 @@ private object ExtractableLiteral { abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { override def buildFilter(predicates: Seq[Expression], cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + // Most of this code originally came from `InMemoryTableScanExec.filteredCachedBatches()` and + // `InMemoryTableScanExec.buildFilter` val stats = new PartitionStatistics(cachedAttributes) val statsSchema = stats.schema 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 8934bef42a8c2..0dac91a944d13 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 @@ -60,6 +60,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { def convertForCacheInternal(cachedPlan: SparkPlan, batchSize: Int, useCompression: Boolean): RDD[CachedBatch] = { + // Most of this code originally came from `CachedRDDBuilder.buildBuffers()` val output = cachedPlan.output cachedPlan.execute().mapPartitionsInternal { rowIterator => new Iterator[DefaultCachedBatch] { @@ -107,6 +108,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } override def supportsColumnar(schema: StructType): Boolean = schema.fields.forall(f => + // This code originally came from `InMemoryTableScanExec.supportsColumnar` f.dataType match { // More types can be supported, but this is to match the original implementation that // only supported primitive types "for ease of review" @@ -120,6 +122,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] = { + // Most of this code originally came from `InMemoryTableScanExec.createAndDecompressColumn` val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled val outputSchema = StructType.fromAttributes(selectedAttributes) val columnIndices = From 38ca74124a5d9552fa83bf228258db7bd58595e7 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 20 Jul 2020 16:53:44 -0500 Subject: [PATCH 10/21] Added in abstraction for Columnar input --- .../sql/columnar/CachedBatchSerializer.scala | 54 ++++++++++--- .../spark/sql/execution/CacheManager.scala | 9 +-- .../apache/spark/sql/execution/Columnar.scala | 23 +++++- .../spark/sql/execution/QueryExecution.scala | 29 ++++++- .../adaptive/AdaptiveSparkPlanExec.scala | 2 +- .../execution/columnar/InMemoryRelation.scala | 76 ++++++++++++++----- .../columnar/InMemoryTableScanExec.scala | 2 +- .../columnar/InMemoryColumnarQuerySuite.scala | 15 +++- 8 files changed, 162 insertions(+), 48 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index b5cd49c2ab0a9..b4544a788ff59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -23,11 +23,11 @@ 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.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith} -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.StorageLevel /** * Basic interface that all cached batches of data must support. This is primarily to allow @@ -48,11 +48,43 @@ trait CachedBatch { @Since("3.1.0") trait CachedBatchSerializer extends Serializable { /** - * Run the given plan and convert its output to a implementation of [[CachedBatch]]. - * @param cachedPlan the plan to run. - * @return the RDD containing the batches of data to cache. + * Can `convertForCacheColumnar()` be called instead of `convertForCache()` for this given + * schema? True if it can and false if it cannot. Columnar input is only supported if the + * plan could produce columnar output. Currently this is mostly supported by input formats + * like parquet and orc, but more operations are likely to be supported soon. + * + * @param schema the schema of the data being stored. + * @return True if columnar input can be supported, else false. */ - def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] + def supportsColumnarInput(schema: Seq[Attribute]): Boolean + + /** + * Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertForCache(input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data. + * This will only be called if `supportsColumnarInput()` returned true for the given schema and + * the plan up to this point would could produce columnar output without modifying it. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertForCacheColumnar(input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] /** * Builds a function that can be used to filter batches prior to being decompressed. @@ -78,17 +110,17 @@ trait CachedBatchSerializer extends Serializable { * @param schema the schema of the data being checked. * @return true if columnar output should be used for this schema, else false. */ - def supportsColumnar(schema: StructType): Boolean + def supportsColumnarOutput(schema: StructType): Boolean /** * Decompress the cached data into a ColumnarBatch. This currently is only used if * `supportsColumnar()` returned true for the associated schema, but there are other checks * that can force row based output. One of the main advantages of doing columnar output over row - * based output is the code generation is more standard and can be combined with code generation - * for downstream operations. + * based output is that the code generation is more standard and can be combined with code + * generation for downstream operations. * @param input the cached batches that should be decompressed. * @param cacheAttributes the attributes of the data in the batch. - * @param selectedAttributes the fields that should be loaded from the data, and the order they + * @param selectedAttributes the fields that should be loaded from the data and the order they * should appear in the output batch. * @param conf the configuration for the job. * @return an RDD of the input cached batches transformed into the ColumnarBatch format. @@ -104,8 +136,8 @@ trait CachedBatchSerializer extends Serializable { * generation is advised. * @param input the cached batches that should be decompressed. * @param cacheAttributes the attributes of the data in the batch. - * @param selectedAttributes the field that should be loaded from the data, and the order they - * should appear in the output batch. + * @param selectedAttributes the field that should be loaded from the data and the order they + * should appear in the output rows. * @param conf the configuration for the job. * @return RDD of the rows that were stored in the cached batches. */ 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 aaceba1165e70..7201026b11b6b 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 @@ -85,9 +85,8 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val qe = sessionWithAqeOff.sessionState.executePlan(planToCache) InMemoryRelation( storageLevel, - qe.executedPlan, - tableName, - optimizedPlan = qe.optimizedPlan) + qe, + tableName) } this.synchronized { @@ -193,9 +192,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val sessionWithAqeOff = getOrCloneSessionWithAqeOff(spark) val newCache = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(cd.plan) - InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = qe.executedPlan), - optimizedPlan = qe.optimizedPlan) + InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) } val recomputedPlan = cd.copy(cachedRepresentation = newCache) this.synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index e01cd8598db0c..73ee5afe12412 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -479,7 +479,10 @@ case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { * Apply any user defined [[ColumnarRule]]s and find the correct place to insert transitions * to/from columnar formatted data. */ -case class ApplyColumnarRulesAndInsertTransitions(conf: SQLConf, columnarRules: Seq[ColumnarRule]) +case class ApplyColumnarRulesAndInsertTransitions( + conf: SQLConf, + columnarRules: Seq[ColumnarRule], + allowColumnarOutput: Boolean) extends Rule[SparkPlan] { /** @@ -508,11 +511,27 @@ case class ApplyColumnarRulesAndInsertTransitions(conf: SQLConf, columnarRules: } } + /** + * Leave the output of the plan unchanged (columnar or row), + * but insert any transitions that are needed for the plan to run correctly. + */ + private def insertRequiredTransitions(plan: SparkPlan): SparkPlan = { + if (plan.supportsColumnar) { + insertRowToColumnar(plan) + } else { + insertTransitions(plan) + } + } + def apply(plan: SparkPlan): SparkPlan = { var preInsertPlan: SparkPlan = plan columnarRules.foreach((r : ColumnarRule) => preInsertPlan = r.preColumnarTransitions(preInsertPlan)) - var postInsertPlan = insertTransitions(preInsertPlan) + var postInsertPlan = if (allowColumnarOutput) { + insertRequiredTransitions(preInsertPlan) + } else { + insertTransitions(preInsertPlan) + } columnarRules.reverse.foreach((r : ColumnarRule) => postInsertPlan = r.postColumnarTransitions(postInsertPlan)) postInsertPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 791e432269632..4dd1e8eea8da3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -97,6 +97,19 @@ class QueryExecution( } } + // a plan like [[executedPlan]] but its output might or might not be columnar. [[executedPlan]] + // guarantees that the resulting plan produces rows. + lazy val maybeColumnarExecutedPlan: SparkPlan = { + // We need to materialize the optimizedPlan here, before tracking the planning phase, to ensure + // that the optimization time is not counted as part of the planning phase. + assertOptimized() + executePhase(QueryPlanningTracker.PLANNING) { + // clone the plan to avoid sharing the plan instance between different stages like analyzing, + // optimizing and planning. + QueryExecution.prepareForExecution(maybeColumnarPreparations, sparkPlan.clone()) + } + } + // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. lazy val executedPlan: SparkPlan = { @@ -126,9 +139,19 @@ class QueryExecution( /** Get the metrics observed during the execution of the query plan. */ def observedMetrics: Map[String, Row] = CollectMetricsExec.collect(executedPlan) + // Similar to [[preparations]] but can be used to produce a plan where might be columnar. + // [[preparations]] guarantees that the resulting plan will produce rows. + protected def maybeColumnarPreparations: Seq[Rule[SparkPlan]] = { + QueryExecution.preparations(sparkSession, + allowColumnar = true, + adaptiveExecutionRule = + Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) + } + protected def preparations: Seq[Rule[SparkPlan]] = { QueryExecution.preparations(sparkSession, - Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) + adaptiveExecutionRule = + Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) } protected def executePhase[T](phase: String)(block: => T): T = sparkSession.withActive { @@ -327,6 +350,7 @@ object QueryExecution { */ private[execution] def preparations( sparkSession: SparkSession, + allowColumnar: Boolean = false, adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None): Seq[Rule[SparkPlan]] = { // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op // as the original plan is hidden behind `AdaptiveSparkPlanExec`. @@ -337,7 +361,8 @@ object QueryExecution { PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, - sparkSession.sessionState.columnarRules), + sparkSession.sessionState.columnarRules, + allowColumnar), CollapseCodegenStages(sparkSession.sessionState.conf), ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 112090640040a..1f1927d0826d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -101,7 +101,7 @@ case class AdaptiveSparkPlanExec( // added by `CoalesceShufflePartitions`. So they must be executed after it. OptimizeSkewedJoin(conf), OptimizeLocalShuffleReader(conf), - ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), + ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules, false), CollapseCodegenStages(conf) ) 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 0dac91a944d13..d9303cc1f6d4f 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType} @@ -51,18 +51,21 @@ case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: * The default implementation of CachedBatchSerializer. */ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { - override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = { - val batchSize = cachedPlan.conf.columnBatchSize - val useCompression = cachedPlan.conf.useCompression - convertForCacheInternal(cachedPlan, batchSize, useCompression) + override def convertForCache(input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + val batchSize = conf.columnBatchSize + val useCompression = conf.useCompression + convertForCacheInternal(input, schema, batchSize, useCompression) } - def convertForCacheInternal(cachedPlan: SparkPlan, + def convertForCacheInternal(input: RDD[InternalRow], + output: Seq[Attribute], batchSize: Int, useCompression: Boolean): RDD[CachedBatch] = { // Most of this code originally came from `CachedRDDBuilder.buildBuffers()` - val output = cachedPlan.output - cachedPlan.execute().mapPartitionsInternal { rowIterator => + input.mapPartitionsInternal { rowIterator => new Iterator[DefaultCachedBatch] { def next(): DefaultCachedBatch = { val columnBuilders = output.map { attribute => @@ -107,7 +110,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } } - override def supportsColumnar(schema: StructType): Boolean = schema.fields.forall(f => + override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f => // This code originally came from `InMemoryTableScanExec.supportsColumnar` f.dataType match { // More types can be supported, but this is to match the original implementation that @@ -176,6 +179,15 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { columnarIterator } } + + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false + + override def convertForCacheColumnar( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = + throw new IllegalStateException("Columnar input is not Supported") } private[sql] @@ -220,12 +232,22 @@ case class CachedRDDBuilder( } private def buildBuffers(): RDD[CachedBatch] = { - val cached = serializer.convertForCache(cachedPlan) - .map { batch => - sizeInBytesStats.add(batch.sizeInBytes) - rowCountStats.add(batch.numRows) - batch - }.persist(storageLevel) + val cb = if (cachedPlan.supportsColumnar) { + serializer.convertForCacheColumnar(cachedPlan.executeColumnar(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } else { + serializer.convertForCache(cachedPlan.execute(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } + val cached = cb.map { batch => + sizeInBytesStats.add(batch.sizeInBytes) + rowCountStats.add(batch.numRows) + batch + }.persist(storageLevel) cached.setName(cachedName) cached } @@ -246,10 +268,16 @@ object InMemoryRelation { def apply( storageLevel: StorageLevel, - child: SparkPlan, - tableName: Option[String], - optimizedPlan: LogicalPlan): InMemoryRelation = { - val cacheBuilder = CachedRDDBuilder(getSerializer(child.conf), storageLevel, child, tableName) + qe: QueryExecution, + tableName: Option[String]): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val serializer = getSerializer(optimizedPlan.conf) + val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) { + qe.maybeColumnarExecutedPlan + } else { + qe.executedPlan + } + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation @@ -270,9 +298,15 @@ object InMemoryRelation { relation } - def apply(cacheBuilder: CachedRDDBuilder, optimizedPlan: LogicalPlan): InMemoryRelation = { + def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val newBuilder = if (cacheBuilder.serializer.supportsColumnarInput(optimizedPlan.output)) { + cacheBuilder.copy(cachedPlan = qe.maybeColumnarExecutedPlan) + } else { + cacheBuilder.copy(cachedPlan = qe.executedPlan) + } val relation = new InMemoryRelation( - cacheBuilder.cachedPlan.output, cacheBuilder, optimizedPlan.outputOrdering) + newBuilder.cachedPlan.output, newBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } 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 a912c4f961a3b..0e9e3f8550442 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 @@ -70,7 +70,7 @@ case class InMemoryTableScanExec( override val supportsColumnar: Boolean = { conf.cacheVectorizedReaderEnabled && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) && - relation.cacheBuilder.serializer.supportsColumnar(relation.schema) + relation.cacheBuilder.serializer.supportsColumnarOutput(relation.schema) } private lazy val columnarInputRDD: RDD[ColumnarBatch] = { 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 8897c150eac62..ddefdd4ae018c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -22,22 +22,29 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.columnar.CachedBatch -import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ class TestCachedBatchSerializer( useCompression: Boolean, batchSize: Int) extends DefaultCachedBatchSerializer { - override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = - convertForCacheInternal(cachedPlan, batchSize, useCompression) + + override def convertForCache(input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + convertForCacheInternal(input, schema, batchSize, useCompression) + } } class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { From c2d00dc9ed77cd5d7e427ccbab8df10e1eefcbf4 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 21 Jul 2020 10:02:25 -0500 Subject: [PATCH 11/21] Fixed formatting --- .../spark/sql/columnar/CachedBatchSerializer.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index b4544a788ff59..e8d8fb8c8685c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -66,7 +66,8 @@ trait CachedBatchSerializer extends Serializable { * @param conf the config for the query. * @return The data converted into a format more suitable for caching. */ - def convertForCache(input: RDD[InternalRow], + def convertForCache( + input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, conf: SQLConf): RDD[CachedBatch] @@ -81,7 +82,8 @@ trait CachedBatchSerializer extends Serializable { * @param conf the config for the query. * @return The data converted into a format more suitable for caching. */ - def convertForCacheColumnar(input: RDD[ColumnarBatch], + def convertForCacheColumnar( + input: RDD[ColumnarBatch], schema: Seq[Attribute], storageLevel: StorageLevel, conf: SQLConf): RDD[CachedBatch] @@ -99,7 +101,8 @@ trait CachedBatchSerializer extends Serializable { * @return a function that takes the partition id and the iterator of batches in the partition. * It returns an iterator of batches that should be decompressed. */ - def buildFilter(predicates: Seq[Expression], + def buildFilter( + predicates: Seq[Expression], cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] /** From 082e883cca80d87f2f0be3d38cac6afc0754de97 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 22 Jul 2020 11:12:41 -0500 Subject: [PATCH 12/21] Added in a test for columanr input and output to cache serializer --- .../columnar/InMemoryColumnarQuerySuite.scala | 96 ++++++++++++++++++- 1 file changed, 94 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index ddefdd4ae018c..8232318bb3b2d 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 @@ -23,15 +23,17 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, In} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, WholeStageCodegenExec} +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ @@ -47,6 +49,78 @@ class TestCachedBatchSerializer( } } +case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { + override def numRows: Int = data.length + override def sizeInBytes: Long = 4 * data.length +} + +/** + * Very simple serializer that only supports a single int column, but does support columnar. + */ +class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true + + override def convertForCache( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def convertForCacheColumnar( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + if (schema.length != 1 || schema.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing $schema") + } + input.map { cb => + val column = cb.column(0) + val data = column.getInts(0, cb.numRows()) + SingleIntCachedBatch(data) + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = true + + override def decompressColumnar( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + if (selectedAttributes.length != 1 || + selectedAttributes.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + "This is for testing") + } + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + val cv = OnHeapColumnVector.allocateColumns(single.numRows, selectedAttributes.toStructType) + val data = single.data + cv(0).putInts(0, data.length, data, 0) + new ColumnarBatch(cv.toArray, single.numRows) + } + } + + override def decompressToRows( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + def ret(index: Int, cb: Iterator[CachedBatch]): Iterator[CachedBatch] = cb + ret + } +} + class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -563,4 +637,22 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { } } } + + test("Columnar Cache Plugin Plan") { + withTempPath { workDir => + val workDirPath = workDir.getAbsolutePath + val input = Seq(100, 200).toDF("count") + input.write.parquet(workDirPath) + val data = spark.read.parquet(workDirPath) + data.createOrReplaceTempView(s"testDataInt") + val storageLevel = MEMORY_ONLY + val plan = spark.sessionState.executePlan(data.logicalPlan).maybeColumnarExecutedPlan + val inMemoryRelation = InMemoryRelation( + new TestSingleIntColumnarCachedBatchSerializer, + storageLevel, plan, None, data.logicalPlan) + + assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) + checkAnswer(inMemoryRelation, data.collect().toSeq) + } + } } From e58783ec28fc839e5559ca8c2888aa54d3f1f5de Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 22 Jul 2020 15:22:04 -0500 Subject: [PATCH 13/21] Addressed review comments --- .../apache/spark/sql/execution/Columnar.scala | 28 +++++------------- .../spark/sql/execution/QueryExecution.scala | 29 ++----------------- .../adaptive/AdaptiveSparkPlanExec.scala | 2 +- .../execution/columnar/InMemoryRelation.scala | 17 +++++++++-- .../columnar/InMemoryColumnarQuerySuite.scala | 3 +- 5 files changed, 27 insertions(+), 52 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index 73ee5afe12412..99ef72ab4e85d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -49,6 +49,11 @@ class ColumnarRule { def postColumnarTransitions: Rule[SparkPlan] = plan => plan } +/** + * A trait that is used as a tag to indicate a transition from columns to rows. + */ +trait ColumnarToRowTransition extends UnaryExecNode + /** * Provides a common executor to translate an [[RDD]] of [[ColumnarBatch]] into an [[RDD]] of * [[InternalRow]]. This is inserted whenever such a transition is determined to be needed. @@ -57,7 +62,7 @@ class ColumnarRule { * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] and * [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations. */ -case class ColumnarToRowExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { +case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition with CodegenSupport { assert(child.supportsColumnar) override def output: Seq[Attribute] = child.output @@ -481,8 +486,7 @@ case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { */ case class ApplyColumnarRulesAndInsertTransitions( conf: SQLConf, - columnarRules: Seq[ColumnarRule], - allowColumnarOutput: Boolean) + columnarRules: Seq[ColumnarRule]) extends Rule[SparkPlan] { /** @@ -511,27 +515,11 @@ case class ApplyColumnarRulesAndInsertTransitions( } } - /** - * Leave the output of the plan unchanged (columnar or row), - * but insert any transitions that are needed for the plan to run correctly. - */ - private def insertRequiredTransitions(plan: SparkPlan): SparkPlan = { - if (plan.supportsColumnar) { - insertRowToColumnar(plan) - } else { - insertTransitions(plan) - } - } - def apply(plan: SparkPlan): SparkPlan = { var preInsertPlan: SparkPlan = plan columnarRules.foreach((r : ColumnarRule) => preInsertPlan = r.preColumnarTransitions(preInsertPlan)) - var postInsertPlan = if (allowColumnarOutput) { - insertRequiredTransitions(preInsertPlan) - } else { - insertTransitions(preInsertPlan) - } + var postInsertPlan = insertTransitions(preInsertPlan) columnarRules.reverse.foreach((r : ColumnarRule) => postInsertPlan = r.postColumnarTransitions(postInsertPlan)) postInsertPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 4dd1e8eea8da3..791e432269632 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -97,19 +97,6 @@ class QueryExecution( } } - // a plan like [[executedPlan]] but its output might or might not be columnar. [[executedPlan]] - // guarantees that the resulting plan produces rows. - lazy val maybeColumnarExecutedPlan: SparkPlan = { - // We need to materialize the optimizedPlan here, before tracking the planning phase, to ensure - // that the optimization time is not counted as part of the planning phase. - assertOptimized() - executePhase(QueryPlanningTracker.PLANNING) { - // clone the plan to avoid sharing the plan instance between different stages like analyzing, - // optimizing and planning. - QueryExecution.prepareForExecution(maybeColumnarPreparations, sparkPlan.clone()) - } - } - // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. lazy val executedPlan: SparkPlan = { @@ -139,19 +126,9 @@ class QueryExecution( /** Get the metrics observed during the execution of the query plan. */ def observedMetrics: Map[String, Row] = CollectMetricsExec.collect(executedPlan) - // Similar to [[preparations]] but can be used to produce a plan where might be columnar. - // [[preparations]] guarantees that the resulting plan will produce rows. - protected def maybeColumnarPreparations: Seq[Rule[SparkPlan]] = { - QueryExecution.preparations(sparkSession, - allowColumnar = true, - adaptiveExecutionRule = - Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) - } - protected def preparations: Seq[Rule[SparkPlan]] = { QueryExecution.preparations(sparkSession, - adaptiveExecutionRule = - Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) + Option(InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession, this)))) } protected def executePhase[T](phase: String)(block: => T): T = sparkSession.withActive { @@ -350,7 +327,6 @@ object QueryExecution { */ private[execution] def preparations( sparkSession: SparkSession, - allowColumnar: Boolean = false, adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None): Seq[Rule[SparkPlan]] = { // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op // as the original plan is hidden behind `AdaptiveSparkPlanExec`. @@ -361,8 +337,7 @@ object QueryExecution { PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, - sparkSession.sessionState.columnarRules, - allowColumnar), + sparkSession.sessionState.columnarRules), CollapseCodegenStages(sparkSession.sessionState.conf), ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 1f1927d0826d0..112090640040a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -101,7 +101,7 @@ case class AdaptiveSparkPlanExec( // added by `CoalesceShufflePartitions`. So they must be executed after it. OptimizeSkewedJoin(conf), OptimizeLocalShuffleReader(conf), - ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules, false), + ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), CollapseCodegenStages(conf) ) 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 d9303cc1f6d4f..929f9685d3468 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} -import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType} @@ -266,6 +266,17 @@ object InMemoryRelation { ser.get } + def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match { + case gen: WholeStageCodegenExec => gen.child match { + case c2r: ColumnarToRowTransition => c2r.child match { + case ia: InputAdapter => ia.child + case _ => plan + } + case _ => plan + } + case _ => plan + } + def apply( storageLevel: StorageLevel, qe: QueryExecution, @@ -273,7 +284,7 @@ object InMemoryRelation { val optimizedPlan = qe.optimizedPlan val serializer = getSerializer(optimizedPlan.conf) val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) { - qe.maybeColumnarExecutedPlan + convertToColumnarIfPossible(qe.executedPlan) } else { qe.executedPlan } @@ -301,7 +312,7 @@ object InMemoryRelation { def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = { val optimizedPlan = qe.optimizedPlan val newBuilder = if (cacheBuilder.serializer.supportsColumnarInput(optimizedPlan.output)) { - cacheBuilder.copy(cachedPlan = qe.maybeColumnarExecutedPlan) + cacheBuilder.copy(cachedPlan = convertToColumnarIfPossible(qe.executedPlan)) } else { cacheBuilder.copy(cachedPlan = qe.executedPlan) } 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 8232318bb3b2d..cb961b636951e 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 @@ -646,7 +646,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { val data = spark.read.parquet(workDirPath) data.createOrReplaceTempView(s"testDataInt") val storageLevel = MEMORY_ONLY - val plan = spark.sessionState.executePlan(data.logicalPlan).maybeColumnarExecutedPlan + val plan = InMemoryRelation.convertToColumnarIfPossible( + spark.sessionState.executePlan(data.logicalPlan).executedPlan) val inMemoryRelation = InMemoryRelation( new TestSingleIntColumnarCachedBatchSerializer, storageLevel, plan, None, data.logicalPlan) From e0f53988633d5038e0d9552255bf1b08299e1d29 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 27 Jul 2020 13:45:50 -0500 Subject: [PATCH 14/21] Fixed issue with columnar type for data generation --- .../spark/sql/columnar/CachedBatchSerializer.scala | 8 ++++++++ .../spark/sql/execution/columnar/InMemoryRelation.scala | 9 +++++++++ .../sql/execution/columnar/InMemoryTableScanExec.scala | 9 +-------- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index e8d8fb8c8685c..4e470c885b1bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -115,6 +115,14 @@ trait CachedBatchSerializer extends Serializable { */ def supportsColumnarOutput(schema: StructType): Boolean + /** + * The exact java types of the columns that are output in columnar processing mode. This + * is a performance optimization for code generation and is optional. + * @param attributes the attributes to be output. + * @param conf the config for the query that will read the data. + */ + def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = None + /** * Decompress the cached data into a ColumnarBatch. This currently is only used if * `supportsColumnar()` returned true for the associated schema, but there are other checks 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 929f9685d3468..a3be67ac1e78f 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 @@ -120,6 +120,15 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { case _ => false }) + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Option(Seq.fill(attributes.length)( + if (!conf.offHeapColumnVectorEnabled) { + classOf[OnHeapColumnVector].getName + } else { + classOf[OffHeapColumnVector].getName + } + )) + override def decompressColumnar( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 0e9e3f8550442..5bc715f023982 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 @@ -28,7 +28,6 @@ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized._ import org.apache.spark.sql.vectorized.ColumnarBatch - case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @@ -55,13 +54,7 @@ case class InMemoryTableScanExec( relation = relation.canonicalized.asInstanceOf[InMemoryRelation]) override def vectorTypes: Option[Seq[String]] = - Option(Seq.fill(attributes.length)( - if (!conf.offHeapColumnVectorEnabled) { - classOf[OnHeapColumnVector].getName - } else { - classOf[OffHeapColumnVector].getName - } - )) + relation.cacheBuilder.serializer.vectorTypes(attributes, conf) /** * If true, get data from ColumnVector in ColumnarBatch, which are generally faster. From c9b1488c84bbd43653ff557667a624597a2037cb Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 28 Jul 2020 13:49:38 -0500 Subject: [PATCH 15/21] Addressed review comments --- .../sql/columnar/CachedBatchSerializer.scala | 17 ++- .../apache/spark/sql/execution/Columnar.scala | 4 +- .../execution/columnar/InMemoryRelation.scala | 33 ++--- .../columnar/InMemoryTableScanExec.scala | 33 +++-- .../columnar/CachedBatchSerializerSuite.scala | 134 ++++++++++++++++++ .../columnar/InMemoryColumnarQuerySuite.scala | 97 +------------ 6 files changed, 180 insertions(+), 138 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 4e470c885b1bb..204363cd0d396 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -124,35 +124,35 @@ trait CachedBatchSerializer extends Serializable { def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = None /** - * Decompress the cached data into a ColumnarBatch. This currently is only used if + * Convert the cached data into a ColumnarBatch. This currently is only used if * `supportsColumnar()` returned true for the associated schema, but there are other checks * that can force row based output. One of the main advantages of doing columnar output over row * based output is that the code generation is more standard and can be combined with code * generation for downstream operations. - * @param input the cached batches that should be decompressed. + * @param input the cached batches that should be converted. * @param cacheAttributes the attributes of the data in the batch. * @param selectedAttributes the fields that should be loaded from the data and the order they * should appear in the output batch. * @param conf the configuration for the job. * @return an RDD of the input cached batches transformed into the ColumnarBatch format. */ - def decompressColumnar( + def convertFromCacheColumnar( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] /** - * Decompress the cached batch into `InternalRow`. If you want this to be performant, code + * Convert the cached batch into `InternalRow`s. If you want this to be performant, code * generation is advised. - * @param input the cached batches that should be decompressed. + * @param input the cached batches that should be converted. * @param cacheAttributes the attributes of the data in the batch. * @param selectedAttributes the field that should be loaded from the data and the order they * should appear in the output rows. * @param conf the configuration for the job. * @return RDD of the rows that were stored in the cached batches. */ - def decompressToRows( + def convertFromCache( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], @@ -204,10 +204,9 @@ private object ExtractableLiteral { @DeveloperApi @Since("3.1.0") abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { - override def buildFilter(predicates: Seq[Expression], + override def buildFilter( + predicates: Seq[Expression], cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { - // Most of this code originally came from `InMemoryTableScanExec.filteredCachedBatches()` and - // `InMemoryTableScanExec.buildFilter` val stats = new PartitionStatistics(cachedAttributes) val statsSchema = stats.schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index 99ef72ab4e85d..13ea609f7bfa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -50,7 +50,9 @@ class ColumnarRule { } /** - * A trait that is used as a tag to indicate a transition from columns to rows. + * A trait that is used as a tag to indicate a transition from columns to rows. This allows plugins + * to replace the current [[ColumnarToRowExec]] with an optimized version and still have operations + * that walk a spark plan looking for this type of transition properly match it. */ trait ColumnarToRowTransition extends UnaryExecNode 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 a3be67ac1e78f..10e2aafde7fd0 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 @@ -51,6 +51,15 @@ case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: * The default implementation of CachedBatchSerializer. */ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false + + override def convertForCacheColumnar( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = + throw new IllegalStateException("Columnar input is not supported") + override def convertForCache(input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, @@ -64,7 +73,6 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { output: Seq[Attribute], batchSize: Int, useCompression: Boolean): RDD[CachedBatch] = { - // Most of this code originally came from `CachedRDDBuilder.buildBuffers()` input.mapPartitionsInternal { rowIterator => new Iterator[DefaultCachedBatch] { def next(): DefaultCachedBatch = { @@ -111,7 +119,6 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f => - // This code originally came from `InMemoryTableScanExec.supportsColumnar` f.dataType match { // More types can be supported, but this is to match the original implementation that // only supported primitive types "for ease of review" @@ -129,12 +136,11 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } )) - override def decompressColumnar( + override def convertFromCacheColumnar( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] = { - // Most of this code originally came from `InMemoryTableScanExec.createAndDecompressColumn` val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled val outputSchema = StructType.fromAttributes(selectedAttributes) val columnIndices = @@ -165,7 +171,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { input.map(createAndDecompressColumn) } - override def decompressToRows(input: RDD[CachedBatch], + override def convertFromCache(input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[InternalRow] = { @@ -188,15 +194,6 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { columnarIterator } } - - override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false - - override def convertForCacheColumnar( - input: RDD[ColumnarBatch], - schema: Seq[Attribute], - storageLevel: StorageLevel, - conf: SQLConf): RDD[CachedBatch] = - throw new IllegalStateException("Columnar input is not Supported") } private[sql] @@ -242,12 +239,14 @@ case class CachedRDDBuilder( private def buildBuffers(): RDD[CachedBatch] = { val cb = if (cachedPlan.supportsColumnar) { - serializer.convertForCacheColumnar(cachedPlan.executeColumnar(), + serializer.convertForCacheColumnar( + cachedPlan.executeColumnar(), cachedPlan.output, storageLevel, cachedPlan.conf) } else { - serializer.convertForCache(cachedPlan.execute(), + serializer.convertForCache( + cachedPlan.execute(), cachedPlan.output, storageLevel, cachedPlan.conf) @@ -283,6 +282,8 @@ object InMemoryRelation { } case _ => plan } + case c2r: ColumnarToRowTransition => // This matches when whole stage code gen is disabled. + c2r.child case _ => plan } 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 5bc715f023982..07b2b165609e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized._ import org.apache.spark.sql.vectorized.ColumnarBatch case class InMemoryTableScanExec( @@ -69,11 +68,14 @@ case class InMemoryTableScanExec( private lazy val columnarInputRDD: RDD[ColumnarBatch] = { val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() - relation.cacheBuilder.serializer.decompressColumnar(buffers, relation.output, attributes, conf) - .map { cb => - numOutputRows += cb.numRows() - cb - } + relation.cacheBuilder.serializer.convertFromCacheColumnar( + buffers, + relation.output, + attributes, + conf).map { cb => + numOutputRows += cb.numRows() + cb + } } private lazy val inputRDD: RDD[InternalRow] = { @@ -89,18 +91,15 @@ case class InMemoryTableScanExec( val serializer = relation.cacheBuilder.serializer // update SQL metrics - val withMetrics = if (enableAccumulatorsForTest) { + val withMetrics = filteredCachedBatches().map{ batch => - readBatches.add(1) + if (enableAccumulatorsForTest) { + readBatches.add(1) + } + numOutputRows += batch.numRows batch } - } else { - filteredCachedBatches() - }.map{ batch => - numOutputRows += batch.numRows - batch - } - val decompressedRows = serializer.decompressToRows(withMetrics, relOutput, attributes, conf) + val rows = serializer.convertFromCache(withMetrics, relOutput, attributes, conf) if (enableAccumulatorsForTest) { def incParts(index: Int, iter: Iterator[InternalRow]): Iterator[InternalRow] = { if (iter.hasNext) { @@ -108,9 +107,9 @@ case class InMemoryTableScanExec( } iter } - decompressedRows.mapPartitionsWithIndexInternal(incParts) + rows.mapPartitionsWithIndexInternal(incParts) } else { - decompressedRows + rows } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala new file mode 100644 index 0000000000000..4a301b28d27fd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala @@ -0,0 +1,134 @@ +/* + * 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.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.storage.StorageLevel + +case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { + override def numRows: Int = data.length + override def sizeInBytes: Long = 4 * data.length +} + +/** + * Very simple serializer that only supports a single int column, but does support columnar. + */ +class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true + + override def convertForCache( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def convertForCacheColumnar( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + if (schema.length != 1 || schema.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing $schema") + } + input.map { cb => + val column = cb.column(0) + val data = column.getInts(0, cb.numRows()) + SingleIntCachedBatch(data) + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = true + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Some(attributes.map(_ => classOf[OnHeapColumnVector].getName)) + + override def convertFromCacheColumnar( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + if (selectedAttributes.isEmpty) { + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + new ColumnarBatch(new Array[ColumnVector](0), single.numRows) + } + } else { + if (selectedAttributes.length > 1 || + selectedAttributes.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing") + } + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + val cv = OnHeapColumnVector.allocateColumns(single.numRows, selectedAttributes.toStructType) + val data = single.data + cv(0).putInts(0, data.length, data, 0) + new ColumnarBatch(cv.toArray, single.numRows) + } + } + } + + override def convertFromCache( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + def ret(index: Int, cb: Iterator[CachedBatch]): Iterator[CachedBatch] = cb + ret + } +} + +class CachedBatchSerializerSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + override protected def sparkConf: SparkConf = { + super.sparkConf.set( + StaticSQLConf.SPARK_CACHE_SERIALIZER.key, + classOf[TestSingleIntColumnarCachedBatchSerializer].getName) + } + + test("Columnar Cache Plugin") { + withTempPath { workDir => + val workDirPath = workDir.getAbsolutePath + val input = Seq(100, 200, 300).toDF("count") + input.write.parquet(workDirPath) + val data = spark.read.parquet(workDirPath) + data.cache() + assert(data.count() == 3) + checkAnswer(data, Row(100) :: Row(200) :: Row(300) :: Nil) + } + } +} 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 cb961b636951e..ddefdd4ae018c 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 @@ -23,17 +23,15 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, In} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, WholeStageCodegenExec} -import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ @@ -49,78 +47,6 @@ class TestCachedBatchSerializer( } } -case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { - override def numRows: Int = data.length - override def sizeInBytes: Long = 4 * data.length -} - -/** - * Very simple serializer that only supports a single int column, but does support columnar. - */ -class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { - override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true - - override def convertForCache( - input: RDD[InternalRow], - schema: Seq[Attribute], - storageLevel: StorageLevel, - conf: SQLConf): RDD[CachedBatch] = { - throw new IllegalStateException("This does not work. This is only for testing") - } - - override def convertForCacheColumnar( - input: RDD[ColumnarBatch], - schema: Seq[Attribute], - storageLevel: StorageLevel, - conf: SQLConf): RDD[CachedBatch] = { - if (schema.length != 1 || schema.head.dataType != IntegerType) { - throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + - s"This is for testing $schema") - } - input.map { cb => - val column = cb.column(0) - val data = column.getInts(0, cb.numRows()) - SingleIntCachedBatch(data) - } - } - - override def supportsColumnarOutput(schema: StructType): Boolean = true - - override def decompressColumnar( - input: RDD[CachedBatch], - cacheAttributes: Seq[Attribute], - selectedAttributes: Seq[Attribute], - conf: SQLConf): RDD[ColumnarBatch] = { - if (selectedAttributes.length != 1 || - selectedAttributes.head.dataType != IntegerType) { - throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + - "This is for testing") - } - input.map { cached => - val single = cached.asInstanceOf[SingleIntCachedBatch] - val cv = OnHeapColumnVector.allocateColumns(single.numRows, selectedAttributes.toStructType) - val data = single.data - cv(0).putInts(0, data.length, data, 0) - new ColumnarBatch(cv.toArray, single.numRows) - } - } - - override def decompressToRows( - input: RDD[CachedBatch], - cacheAttributes: Seq[Attribute], - selectedAttributes: Seq[Attribute], - conf: SQLConf): RDD[InternalRow] = { - throw new IllegalStateException("This does not work. This is only for testing") - } - - override def buildFilter( - predicates: Seq[Expression], - cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { - def ret(index: Int, cb: Iterator[CachedBatch]): Iterator[CachedBatch] = cb - ret - } -} - class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -637,23 +563,4 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { } } } - - test("Columnar Cache Plugin Plan") { - withTempPath { workDir => - val workDirPath = workDir.getAbsolutePath - val input = Seq(100, 200).toDF("count") - input.write.parquet(workDirPath) - val data = spark.read.parquet(workDirPath) - data.createOrReplaceTempView(s"testDataInt") - val storageLevel = MEMORY_ONLY - val plan = InMemoryRelation.convertToColumnarIfPossible( - spark.sessionState.executePlan(data.logicalPlan).executedPlan) - val inMemoryRelation = InMemoryRelation( - new TestSingleIntColumnarCachedBatchSerializer, - storageLevel, plan, None, data.logicalPlan) - - assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) - checkAnswer(inMemoryRelation, data.collect().toSeq) - } - } } From 4ec28bba9507440550f23d1af5b4f636ff643661 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 28 Jul 2020 16:55:11 -0500 Subject: [PATCH 16/21] Addressed more comments --- .../sql/columnar/CachedBatchSerializer.scala | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 204363cd0d396..51c652f805bd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -125,7 +125,7 @@ trait CachedBatchSerializer extends Serializable { /** * Convert the cached data into a ColumnarBatch. This currently is only used if - * `supportsColumnar()` returned true for the associated schema, but there are other checks + * `supportsColumnar()` returns true for the associated schema, but there are other checks * that can force row based output. One of the main advantages of doing columnar output over row * based output is that the code generation is more standard and can be combined with code * generation for downstream operations. @@ -169,16 +169,24 @@ trait CachedBatchSerializer extends Serializable { @Since("3.1.0") trait SimpleMetricsCachedBatch extends CachedBatch { /** - * Holds the same as ColumnStats. - * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long - * Which is repeated for each column in the original data. + * Holds stats for each cached column. The optional `upperBound` and `lowerBound` should be + * of the same type as the original column. If they are null, then it is assumed that they + * are not provided, and will not be used for filtering. + *
    + *
  • `upperBound` (optional)<\li> + *
  • `lowerBound` (Optional)<\li> + *
  • `nullCount`: `Int`<\li> + *
  • `rowCount`: `Int`
  • + *
  • `sizeInBytes`: `Long`
  • + *
      + * These are repeated for each column in the original cached data. */ val stats: InternalRow override def sizeInBytes: Long = Range.apply(4, stats.numFields, 5).map(stats.getLong).sum } -// Currently, only use statistics from atomic types except binary type only. +// Currently, uses statistics for all atomic types that are not `BinaryType`. private object ExtractableLiteral { def unapply(expr: Expression): Option[Literal] = expr match { case lit: Literal => lit.dataType match { From 7840a4c325813f7b027a3d823098b72eed108eab Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 29 Jul 2020 09:10:35 -0500 Subject: [PATCH 17/21] Fixed docs --- .../apache/spark/sql/columnar/CachedBatchSerializer.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 51c652f805bd9..ba2c7ae11b408 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -173,12 +173,12 @@ trait SimpleMetricsCachedBatch extends CachedBatch { * of the same type as the original column. If they are null, then it is assumed that they * are not provided, and will not be used for filtering. *
        - *
      • `upperBound` (optional)<\li> - *
      • `lowerBound` (Optional)<\li> - *
      • `nullCount`: `Int`<\li> + *
      • `upperBound` (optional)
      • + *
      • `lowerBound` (Optional)
      • + *
      • `nullCount`: `Int`
      • *
      • `rowCount`: `Int`
      • *
      • `sizeInBytes`: `Long`
      • - *
          + *
        * These are repeated for each column in the original cached data. */ val stats: InternalRow From 5723aa9b5418e0ded1d160e68e88f8b43963797a Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 29 Jul 2020 09:38:35 -0500 Subject: [PATCH 18/21] Addressed review comments --- .../sql/columnar/CachedBatchSerializer.scala | 27 ++++++++++--------- .../execution/columnar/InMemoryRelation.scala | 18 ++++++------- .../columnar/InMemoryTableScanExec.scala | 4 +-- .../columnar/CachedBatchSerializerSuite.scala | 8 +++--- .../columnar/InMemoryColumnarQuerySuite.scala | 2 +- 5 files changed, 30 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index ba2c7ae11b408..4b46cf1435da9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -48,11 +48,11 @@ trait CachedBatch { @Since("3.1.0") trait CachedBatchSerializer extends Serializable { /** - * Can `convertForCacheColumnar()` be called instead of `convertForCache()` for this given - * schema? True if it can and false if it cannot. Columnar input is only supported if the - * plan could produce columnar output. Currently this is mostly supported by input formats - * like parquet and orc, but more operations are likely to be supported soon. - * + * Can `convertColumnarBatchToCachedBatch()` be called instead of + * `convertInternalRowToCachedBatch()` for this given schema? True if it can and false if it + * cannot. Columnar input is only supported if the plan could produce columnar output. Currently + * this is mostly supported by input formats like parquet and orc, but more operations are likely + * to be supported soon. * @param schema the schema of the data being stored. * @return True if columnar input can be supported, else false. */ @@ -66,7 +66,7 @@ trait CachedBatchSerializer extends Serializable { * @param conf the config for the query. * @return The data converted into a format more suitable for caching. */ - def convertForCache( + def convertInternalRowToCachedBatch( input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, @@ -82,7 +82,7 @@ trait CachedBatchSerializer extends Serializable { * @param conf the config for the query. * @return The data converted into a format more suitable for caching. */ - def convertForCacheColumnar( + def convertColumnarBatchToCachedBatch( input: RDD[ColumnarBatch], schema: Seq[Attribute], storageLevel: StorageLevel, @@ -106,10 +106,11 @@ trait CachedBatchSerializer extends Serializable { cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] /** - * Can `decompressColumnar()` be called instead of `decompressToRows()` for this given - * schema? True if it can and false if it cannot. Columnar output is typically preferred - * because it is more efficient. Note that `decompressToRows()` must always be supported - * as there are other checks that can force row based output. + * Can `convertCachedBatchToColumnarBatch()` be called instead of + * `convertCachedBatchToInternalRow()` for this given schema? True if it can and false if it + * cannot. Columnar output is typically preferred because it is more efficient. Note that + * `convertCachedBatchToInternalRow()` must always be supported as there are other checks that + * can force row based output. * @param schema the schema of the data being checked. * @return true if columnar output should be used for this schema, else false. */ @@ -136,7 +137,7 @@ trait CachedBatchSerializer extends Serializable { * @param conf the configuration for the job. * @return an RDD of the input cached batches transformed into the ColumnarBatch format. */ - def convertFromCacheColumnar( + def convertCachedBatchToColumnarBatch( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], @@ -152,7 +153,7 @@ trait CachedBatchSerializer extends Serializable { * @param conf the configuration for the job. * @return RDD of the rows that were stored in the cached batches. */ - def convertFromCache( + def convertCachedBatchToInternalRow( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], 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 10e2aafde7fd0..0c22c7ab60d9a 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 @@ -45,7 +45,7 @@ import org.apache.spark.util.{LongAccumulator, Utils} * @param stats The stat of columns */ case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) - extends SimpleMetricsCachedBatch + extends SimpleMetricsCachedBatch /** * The default implementation of CachedBatchSerializer. @@ -53,14 +53,14 @@ case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false - override def convertForCacheColumnar( + override def convertColumnarBatchToCachedBatch( input: RDD[ColumnarBatch], schema: Seq[Attribute], storageLevel: StorageLevel, conf: SQLConf): RDD[CachedBatch] = throw new IllegalStateException("Columnar input is not supported") - override def convertForCache(input: RDD[InternalRow], + override def convertInternalRowToCachedBatch(input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, conf: SQLConf): RDD[CachedBatch] = { @@ -120,8 +120,8 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f => f.dataType match { - // More types can be supported, but this is to match the original implementation that - // only supported primitive types "for ease of review" + // More types can be supported, but this is to match the original implementation that + // only supported primitive types "for ease of review" case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType => true case _ => false @@ -136,7 +136,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } )) - override def convertFromCacheColumnar( + override def convertCachedBatchToColumnarBatch( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], @@ -171,7 +171,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { input.map(createAndDecompressColumn) } - override def convertFromCache(input: RDD[CachedBatch], + override def convertCachedBatchToInternalRow(input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[InternalRow] = { @@ -239,13 +239,13 @@ case class CachedRDDBuilder( private def buildBuffers(): RDD[CachedBatch] = { val cb = if (cachedPlan.supportsColumnar) { - serializer.convertForCacheColumnar( + serializer.convertColumnarBatchToCachedBatch( cachedPlan.executeColumnar(), cachedPlan.output, storageLevel, cachedPlan.conf) } else { - serializer.convertForCache( + serializer.convertInternalRowToCachedBatch( cachedPlan.execute(), cachedPlan.output, storageLevel, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 07b2b165609e8..cde70e5a7937e 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 @@ -68,7 +68,7 @@ case class InMemoryTableScanExec( private lazy val columnarInputRDD: RDD[ColumnarBatch] = { val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() - relation.cacheBuilder.serializer.convertFromCacheColumnar( + relation.cacheBuilder.serializer.convertCachedBatchToColumnarBatch( buffers, relation.output, attributes, @@ -99,7 +99,7 @@ case class InMemoryTableScanExec( numOutputRows += batch.numRows batch } - val rows = serializer.convertFromCache(withMetrics, relOutput, attributes, conf) + val rows = serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) if (enableAccumulatorsForTest) { def incParts(index: Int, iter: Iterator[InternalRow]): Iterator[InternalRow] = { if (iter.hasNext) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala index 4a301b28d27fd..72eba7f6e6907 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala @@ -41,7 +41,7 @@ case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true - override def convertForCache( + override def convertInternalRowToCachedBatch( input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, @@ -49,7 +49,7 @@ class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { throw new IllegalStateException("This does not work. This is only for testing") } - override def convertForCacheColumnar( + override def convertColumnarBatchToCachedBatch( input: RDD[ColumnarBatch], schema: Seq[Attribute], storageLevel: StorageLevel, @@ -69,7 +69,7 @@ class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = Some(attributes.map(_ => classOf[OnHeapColumnVector].getName)) - override def convertFromCacheColumnar( + override def convertCachedBatchToColumnarBatch( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], @@ -95,7 +95,7 @@ class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { } } - override def convertFromCache( + override def convertCachedBatchToInternalRow( input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], 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 ddefdd4ae018c..b8f73f4563ef1 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 @@ -39,7 +39,7 @@ class TestCachedBatchSerializer( useCompression: Boolean, batchSize: Int) extends DefaultCachedBatchSerializer { - override def convertForCache(input: RDD[InternalRow], + override def convertInternalRowToCachedBatch(input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, conf: SQLConf): RDD[CachedBatch] = { From f8ca4afa41cc2139ff2ad345396b235719235b23 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 30 Jul 2020 08:39:19 -0500 Subject: [PATCH 19/21] Addressed review comments --- .../sql/columnar/CachedBatchSerializer.scala | 3 +- .../execution/columnar/InMemoryRelation.scala | 9 ++++-- .../columnar/InMemoryTableScanExec.scala | 31 +++++++++---------- 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 4b46cf1435da9..3b8de62f1a0f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -41,8 +41,7 @@ trait CachedBatch { } /** - * Provides APIs for compressing, filtering, and decompressing SQL data that will be - * persisted/cached. + * Provides APIs that handle transformations of SQL data associated with the cache/persist APIs. */ @DeveloperApi @Since("3.1.0") 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 0c22c7ab60d9a..be3dc5934e84f 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 @@ -60,7 +60,8 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { conf: SQLConf): RDD[CachedBatch] = throw new IllegalStateException("Columnar input is not supported") - override def convertInternalRowToCachedBatch(input: RDD[InternalRow], + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], schema: Seq[Attribute], storageLevel: StorageLevel, conf: SQLConf): RDD[CachedBatch] = { @@ -69,7 +70,8 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { convertForCacheInternal(input, schema, batchSize, useCompression) } - def convertForCacheInternal(input: RDD[InternalRow], + def convertForCacheInternal( + input: RDD[InternalRow], output: Seq[Attribute], batchSize: Int, useCompression: Boolean): RDD[CachedBatch] = { @@ -171,7 +173,8 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { input.map(createAndDecompressColumn) } - override def convertCachedBatchToInternalRow(input: RDD[CachedBatch], + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[InternalRow] = { 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 cde70e5a7937e..ad594d3370f48 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 @@ -92,25 +92,24 @@ case class InMemoryTableScanExec( // update SQL metrics val withMetrics = - filteredCachedBatches().map{ batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) - } - numOutputRows += batch.numRows - batch - } - val rows = serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) - if (enableAccumulatorsForTest) { - def incParts(index: Int, iter: Iterator[InternalRow]): Iterator[InternalRow] = { - if (iter.hasNext) { + filteredCachedBatches().mapPartitionsInternal { iter => + if (enableAccumulatorsForTest && iter.hasNext) { readPartitions.add(1) } - iter + new Iterator[CachedBatch] { + override def hasNext: Boolean = iter.hasNext + + override def next(): CachedBatch = { + val batch = iter.next + if (enableAccumulatorsForTest) { + readBatches.add(1) + } + numOutputRows += batch.numRows + batch + } + } } - rows.mapPartitionsWithIndexInternal(incParts) - } else { - rows - } + serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) } override def output: Seq[Attribute] = attributes From ea762e596b4653227835fed27f3093868b6c8d72 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 30 Jul 2020 09:16:43 -0500 Subject: [PATCH 20/21] Addressed review comments --- .../columnar/InMemoryTableScanExec.scala | 15 +++++---------- 1 file changed, 5 insertions(+), 10 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 ad594d3370f48..e4194562b7a5d 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 @@ -96,17 +96,12 @@ case class InMemoryTableScanExec( if (enableAccumulatorsForTest && iter.hasNext) { readPartitions.add(1) } - new Iterator[CachedBatch] { - override def hasNext: Boolean = iter.hasNext - - override def next(): CachedBatch = { - val batch = iter.next - if (enableAccumulatorsForTest) { - readBatches.add(1) - } - numOutputRows += batch.numRows - batch + iter.map { batch => + if (enableAccumulatorsForTest) { + readBatches.add(1) } + numOutputRows += batch.numRows + batch } } serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) From 3f2f5278c7e8c05e32a5f54ddba1251250dfdca3 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 31 Jul 2020 12:38:31 -0500 Subject: [PATCH 21/21] Addressed review comments --- .../org/apache/spark/sql/columnar/CachedBatchSerializer.scala | 2 +- .../org/apache/spark/sql/execution/columnar/ColumnStats.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 3b8de62f1a0f6..1113e63cab332 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -125,7 +125,7 @@ trait CachedBatchSerializer extends Serializable { /** * Convert the cached data into a ColumnarBatch. This currently is only used if - * `supportsColumnar()` returns true for the associated schema, but there are other checks + * `supportsColumnarOutput()` returns true for the associated schema, but there are other checks * that can force row based output. One of the main advantages of doing columnar output over row * based output is that the code generation is more standard and can be combined with code * generation for downstream operations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index ba7bf9119cbbf..45557bfbada6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { +class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() @@ -32,7 +32,7 @@ private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } -private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { +class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) (AttributeMap(allStats), allStats.flatMap(_._2.schema))