cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r461677607
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar
import org.apache.commons.lang3.StringUtils
+import org.apache.spark.TaskContext
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, 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.columnar.{CachedBatch, CachedBatchSerializer,
SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+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}
+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}
/**
- * CachedBatch is a cached batch of rows.
+ * 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
*/
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats:
InternalRow)
+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(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(input: RDD[InternalRow],
+ output: Seq[Attribute],
+ batchSize: Int,
+ useCompression: Boolean): RDD[CachedBatch] = {
+ // Most of this code originally came from `CachedRDDBuilder.buildBuffers()`
Review comment:
we should remove it (and other similar comments) before PR merging.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ val cb = if (cachedPlan.supportsColumnar) {
+ serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
Review comment:
super nit:
```
obj.func(
para1,
para2,
...)
```
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ 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
}
}
object InMemoryRelation {
+ private[this] var ser: Option[CachedBatchSerializer] = None
+ private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer =
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 convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+ case gen: WholeStageCodegenExec => gen.child match {
+ case c2r: ColumnarToRowTransition => c2r.child match {
+ case ia: InputAdapter => ia.child
Review comment:
hmm, why do we expect `InputAdapter`? Is it possible to be
`ColumnarToRowExec(ColumnarProjectExec(...))`?
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.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
+ * 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 {
+ /**
+ * 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 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.
+ * 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 decompressed.
+ */
+ def buildFilter(
+ predicates: Seq[Expression],
+ 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.
+ * @param schema the schema of the data being checked.
+ * @return true if columnar output should be used for this schema, else
false.
+ */
+ 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
+ * 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 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(
Review comment:
the name `decompress` sounds like the cache must be compressed, which
may not be true. How about `readCachedData` and `readCachedDataColumnar`?
##########
File path:
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
##########
@@ -551,4 +637,23 @@ class InMemoryColumnarQuerySuite extends QueryTest with
SharedSparkSession {
}
}
}
+
+ test("Columnar Cache Plugin Plan") {
Review comment:
Can we move it to a new test suite? Then we can set static conf in that
suite to create the spark session, and have a real end-to-end test.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +85,32 @@ 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) {
Review comment:
shall we keep the original code style, and put `if
(enableAccumulatorsForTest)` inside `filteredCachedBatches().map`? Then we just
need a single `map` call.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar
import org.apache.commons.lang3.StringUtils
+import org.apache.spark.TaskContext
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, 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.columnar.{CachedBatch, CachedBatchSerializer,
SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+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}
+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}
/**
- * CachedBatch is a cached batch of rows.
+ * 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
*/
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats:
InternalRow)
+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(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(input: RDD[InternalRow],
+ 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 = {
+ val columnBuilders = output.map { attribute =>
+ ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
+ }.toArray
+
+ var rowCount = 0
+ var totalSize = 0L
+ while (rowIterator.hasNext && rowCount < batchSize
+ && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
+ val row = rowIterator.next()
+
+ // Added for SPARK-6082. This assertion can be useful for
scenarios when something
+ // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
+ // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
+ // hard to decipher.
+ assert(
+ row.numFields == columnBuilders.length,
+ s"Row column number mismatch, expected ${output.size} columns, "
+
+ s"but got ${row.numFields}." +
+ s"\nRow content: $row")
+
+ var i = 0
+ totalSize = 0
+ while (i < row.numFields) {
+ columnBuilders(i).appendFrom(row, i)
+ totalSize += columnBuilders(i).columnStats.sizeInBytes
+ i += 1
+ }
+ rowCount += 1
+ }
+
+ val stats = InternalRow.fromSeq(
+ columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
+ DefaultCachedBatch(rowCount, columnBuilders.map { builder =>
+ JavaUtils.bufferToArray(builder.build())
+ }, stats)
+ }
+
+ def hasNext: Boolean = rowIterator.hasNext
+ }
+ }
+ }
+
+ 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"
+ case BooleanType | ByteType | ShortType | IntegerType | LongType |
+ FloatType | DoubleType => true
+ 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],
+ 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 =
+ 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
+ }
+ }
+
+ override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false
Review comment:
nit: we should move it to near `convertForCache`
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.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
+ * 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 {
+ /**
+ * 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 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.
+ * 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 decompressed.
+ */
+ def buildFilter(
+ predicates: Seq[Expression],
+ 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.
+ * @param schema the schema of the data being checked.
+ * @return true if columnar output should be used for this schema, else
false.
+ */
+ 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
+ * 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 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(
+ 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 rows.
+ * @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],
Review comment:
super nit:
```
def func(
para1: T,
para2: T): R = ...
```
##########
File path: 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.
Review comment:
maybe explain more about why we need it: help us to know if a query plan
can produce columnar data natively.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ val cb = if (cachedPlan.supportsColumnar) {
+ serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+ cachedPlan.output,
+ storageLevel,
+ cachedPlan.conf)
+ } else {
+ serializer.convertForCache(cachedPlan.execute(),
Review comment:
ditto
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ 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
}
}
object InMemoryRelation {
+ private[this] var ser: Option[CachedBatchSerializer] = None
+ private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer =
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 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
Review comment:
shall we allow whole-stage-codegen disabled? I think it's simple to do so
```
case gen: WholeStageCodegenExec => convertToColumnarIfPossible(gen)
case ... // the real code
```
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ 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
}
}
object InMemoryRelation {
+ private[this] var ser: Option[CachedBatchSerializer] = None
+ private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer =
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 convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+ case gen: WholeStageCodegenExec => gen.child match {
+ case c2r: ColumnarToRowTransition => c2r.child match {
+ case ia: InputAdapter => ia.child
Review comment:
ah I see, so the case here is simply whole-stage-codegened parquet/orc
scan.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ 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
}
}
object InMemoryRelation {
+ private[this] var ser: Option[CachedBatchSerializer] = None
+ private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer =
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 convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+ case gen: WholeStageCodegenExec => gen.child match {
+ case c2r: ColumnarToRowTransition => c2r.child match {
+ case ia: InputAdapter => ia.child
Review comment:
ah I see, so the case here is simply whole-stage-codegened parquet/orc
scan or other columnar operators.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ 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
}
}
object InMemoryRelation {
+ private[this] var ser: Option[CachedBatchSerializer] = None
+ private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer =
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 convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+ case gen: WholeStageCodegenExec => gen.child match {
+ case c2r: ColumnarToRowTransition => c2r.child match {
+ case ia: InputAdapter => ia.child
Review comment:
ah I see, so the case here is a simple `WholeStageCodegenExec` with
parquet/orc scan or other columnar operators.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
}
private def buildBuffers(): RDD[CachedBatch] = {
- val output = cachedPlan.output
- val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
- new Iterator[CachedBatch] {
- def next(): CachedBatch = {
- val columnBuilders = output.map { attribute =>
- ColumnBuilder(attribute.dataType, batchSize, attribute.name,
useCompression)
- }.toArray
-
- var rowCount = 0
- var totalSize = 0L
- while (rowIterator.hasNext && rowCount < batchSize
- && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
- val row = rowIterator.next()
-
- // Added for SPARK-6082. This assertion can be useful for
scenarios when something
- // like Hive TRANSFORM is used. The external data generation
script used in TRANSFORM
- // may result malformed rows, causing
ArrayIndexOutOfBoundsException, which is somewhat
- // hard to decipher.
- assert(
- row.numFields == columnBuilders.length,
- s"Row column number mismatch, expected ${output.size} columns, "
+
- s"but got ${row.numFields}." +
- s"\nRow content: $row")
-
- var i = 0
- totalSize = 0
- while (i < row.numFields) {
- columnBuilders(i).appendFrom(row, i)
- totalSize += columnBuilders(i).columnStats.sizeInBytes
- i += 1
- }
- rowCount += 1
- }
-
- sizeInBytesStats.add(totalSize)
- rowCountStats.add(rowCount)
-
- val stats = InternalRow.fromSeq(
- columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
- CachedBatch(rowCount, columnBuilders.map { builder =>
- JavaUtils.bufferToArray(builder.build())
- }, stats)
- }
-
- def hasNext: Boolean = rowIterator.hasNext
- }
+ 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
}
}
object InMemoryRelation {
+ private[this] var ser: Option[CachedBatchSerializer] = None
+ private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer =
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 convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+ case gen: WholeStageCodegenExec => gen.child match {
+ case c2r: ColumnarToRowTransition => c2r.child match {
+ case ia: InputAdapter => ia.child
Review comment:
ah I see, so the case here is a simple `WholeStageCodegenExec` with
parquet/orc scan or other columnar operators, wrapped by `InputAdapter`.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.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
+ * 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 {
+ /**
+ * 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 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.
+ * 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 decompressed.
+ */
+ def buildFilter(
+ predicates: Seq[Expression],
+ 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.
+ * @param schema the schema of the data being checked.
+ * @return true if columnar output should be used for this schema, else
false.
+ */
+ 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
+ * 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 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(
Review comment:
SGTM, it's also consistent with `convertToCache`
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]