revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r461708141
##########
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:
Code generation is not currently supported for Columnar data except to
transform it into rows before doing more code generation.
##########
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:
Right now yes. Once we have more operators that support columnar
processing it can change.
##########
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:
How about `convertFromCache`? We are not actually reading data from
anywhere but memory, so I am a little reluctant to say it is a read.
##########
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:
Sure I can make the change I did it because I thought it would be more
efficient in the common case when we are not testing, but I could be wrong. My
knowledge of the JVM JIT + Scala compilation is a little lacking in this area.
I know that with Java a `static final boolean` will be inlined during JIT and
checks for that value along with dead code paths are entirely removed.
```
lazy val enableAccumulatorsForTest: Boolean = ...
```
I believe is translated into a function in Scala so I don't think that the
JVM JIT is able to do the same thing, but we are creating a closure for the map
so that might allow it to inline it.
Either way it is `O(number of batches)` so it should be much less of an
issue compared to a row iterator.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,343 @@
+/*
+ * 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
Review comment:
Typically yes, but it could be a custom implementation, like what
Parquet and Orc do.
----------------------------------------------------------------
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]