Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add Parquet-based cache serializer #638

Merged
merged 18 commits into from
Sep 9, 2020
Merged
Show file tree
Hide file tree
Changes from 9 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion integration_tests/src/main/python/cache_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ def do_join(spark):
TimestampGen()]

@pytest.mark.parametrize('data_gen', all_gen_restricting_dates, ids=idfn)
@allow_non_gpu('InMemoryTableScanExec', 'DataWritingCommandExec')
@allow_non_gpu('DataWritingCommandExec')
def test_cache_posexplode_makearray(spark_tmp_path, data_gen):
if data_gen.data_type == BooleanType():
pytest.xfail("https://github.com/NVIDIA/spark-rapids/issues/350")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -359,6 +359,10 @@ class Spark300Shims extends SparkShims {

override def copyFileSourceScanExec(scanExec: GpuFileSourceScanExec,
supportsSmallFileOpt: Boolean): GpuFileSourceScanExec = {
scanExec.copy(supportsSmallFileOpt=supportsSmallFileOpt)
scanExec.copy(supportsSmallFileOpt = supportsSmallFileOpt)
}

override def getGpuColumnarToRowTransition(plan: SparkPlan): String = {
revans2 marked this conversation as resolved.
Show resolved Hide resolved
classOf[GpuColumnarToRowExec].getName
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -108,4 +108,8 @@ class Spark301Shims extends Spark300Shims {
ruleBuilder: SparkSession => Rule[SparkPlan]): Unit = {
extensions.injectQueryStagePrepRule(ruleBuilder)
}

revans2 marked this conversation as resolved.
Show resolved Hide resolved
override def getGpuColumnarToRowTransition(plan: SparkPlan): String = {
classOf[GpuColumnarToRowExec].getName
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,303 @@
/*
* Copyright (c) 2019-2020, NVIDIA CORPORATION.
revans2 marked this conversation as resolved.
Show resolved Hide resolved
*
* Licensed 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 com.nvidia.spark.rapids.shims.spark310

import scala.collection.mutable

import ai.rapids.cudf._
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
import com.nvidia.spark.rapids.GpuMetricNames.{NUM_INPUT_ROWS, NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME}
import com.nvidia.spark.rapids.RapidsPluginImplicits._

import org.apache.spark.TaskContext
import org.apache.spark.rdd.RDD
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.metric.SQLMetrics
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StructField, StructType}
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.storage.StorageLevel

class ParquetCachedBatch(val numRows: Int) extends CachedBatch with HostBufferConsumer
revans2 marked this conversation as resolved.
Show resolved Hide resolved
with AutoCloseable with Serializable {
@transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
private[this] var buffer: Array[Byte] = null
private var bytes: Long = 0

override def sizeInBytes: Long = {
if (bytes == 0) {
val toProcess = offHeapBuffers.dequeueAll(_ => true)
revans2 marked this conversation as resolved.
Show resolved Hide resolved
bytes = toProcess.unzip._2.sum
}
bytes
}

override def handleBuffer(hostMemoryBuffer: HostMemoryBuffer, len: Long): Unit = {
offHeapBuffers += Tuple2(hostMemoryBuffer, len)
bytes += len
}

def getBuffer(): Array[Byte] = {
if (buffer == null) {
writeBuffers()
}
buffer
}

def close(): Unit = {
writeBuffers()
}

def writeBuffers(): Unit = {
// this could be problematic if the buffers are big as their cumulative length could be more
// than an Int.MAX_SIZE. We could just have a list of buffers in that case and iterate over them
revans2 marked this conversation as resolved.
Show resolved Hide resolved
buffer = new Array(sizeInBytes.toInt)
val toProcess = offHeapBuffers.dequeueAll(_ => true)
try {
var offset: Int = 0
toProcess.foreach(ops => {
val origBuffer = ops._1
val len = ops._2.toInt
origBuffer.asByteBuffer().get(buffer, offset, len)
for (i <- 0 until len) {
assert(origBuffer.getByte(i) == buffer(offset + i.toInt))
}
offset = offset + len
})
} finally {
toProcess.map(_._1).safeClose()
}
}
}

private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
revans2 marked this conversation as resolved.
Show resolved Hide resolved
Iterator[ColumnarBatch] {
var cb: ColumnarBatch = null

private def closeCurrentBatch(): Unit = {
if (cb != null) {
cb.close
cb = null
}
}

TaskContext.get().addTaskCompletionListener[Unit]((tc: TaskContext) => {
closeCurrentBatch()
})

override def hasNext: Boolean = iter.hasNext

override def next(): ColumnarBatch = {
closeCurrentBatch()
cb = iter.next()
cb
}
}

/**
* This class assumes, the data is Columnar and the plugin is on
*/
class ParquetCachedBatchSerializer extends CachedBatchSerializer with Arm {
override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true

override def supportsColumnarOutput(schema: StructType): Boolean = true

/**
* Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data.
* This method uses Parquet Writer on the GPU to write the cached batch
* @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.
*/
override def convertColumnarBatchToCachedBatch(input: RDD[ColumnarBatch],
schema: Seq[Attribute],
storageLevel: StorageLevel,
conf: SQLConf): RDD[CachedBatch] = {
val parquetCB = input.map(batch => {
var gpuCB: ColumnarBatch = null
revans2 marked this conversation as resolved.
Show resolved Hide resolved
try {
// check if the data is already on GPU
if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
revans2 marked this conversation as resolved.
Show resolved Hide resolved
val s = StructType(
schema.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows(), batch).build(batch.numRows())
batch.close()
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Technically if it is on the CPU then it is the responsibility of the producer to close the batch, not the consumer.

This is one of the odd things with spark. They expect the producer to close the batch, where as we expect the consumer to do it. They did this so that they could reuse the memory for a batch whenever possible, mostly because it is used in just a limited number of locations and with a very limited size of a batch.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you saying not to close it? If it was being closed by the producer, shouldn't it have caused a doublefree error?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this code is ever tested, but it would probably result in the CPU parquet writer trying to use freed memory.

} else {
gpuCB = batch
}
// now compress it using ParquetWriter
compressColumnarBatchWithParquet(gpuCB)
} finally {
gpuCB.close()
}
})
parquetCB
}

private def compressColumnarBatchWithParquet(gpuCB: ColumnarBatch) = {
val buffer = new ParquetCachedBatch(gpuCB.numRows())
withResource(GpuColumnVector.from(gpuCB)) { table =>
withResource(Table.writeParquetChunked(ParquetWriterOptions.DEFAULT, buffer)) { writer =>
writer.write(table)
}
}
buffer.writeBuffers()
buffer.asInstanceOf[CachedBatch]
}

/**
* This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying back to
* the host
* @param input
* @param cacheAttributes
* @param selectedAttributes
* @param conf
* @return
revans2 marked this conversation as resolved.
Show resolved Hide resolved
*/
def gpuConvertCachedBatchToColumnarBatch(input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
conf: SQLConf): RDD[ColumnarBatch] = {
val cbRdd = convertCachedBatchToColumnarInternal(input, cacheAttributes, selectedAttributes)
revans2 marked this conversation as resolved.
Show resolved Hide resolved
cbRdd
}

private def convertCachedBatchToColumnarInternal(input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute]) = {
val requestedColumnIndices = selectedAttributes.map(a =>
cacheAttributes.map(_.exprId).indexOf(a.exprId))

// if plugin enabled
revans2 marked this conversation as resolved.
Show resolved Hide resolved
val cbRdd: RDD[ColumnarBatch] = input.map(batch => {
if (batch.isInstanceOf[ParquetCachedBatch]) {
val parquetCB = batch.asInstanceOf[ParquetCachedBatch]
withResource(Table.readParquet(ParquetOptions.DEFAULT, parquetCB.getBuffer(), 0,
revans2 marked this conversation as resolved.
Show resolved Hide resolved
parquetCB.sizeInBytes)) { table =>
withResource(GpuColumnVector.from(table)) { cb =>
val cols = GpuColumnVector.extractColumns(cb)
new ColumnarBatch(requestedColumnIndices.map(ordinal =>
cols(ordinal).incRefCount()).toArray, cb.numRows())
}
}
} else {
throw new IllegalStateException("I don't know how to convert this batch")
}
})
cbRdd
}

/**
* Convert the cached data into a ColumnarBatch taking the result data back to the host
* @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.
*/
override def convertCachedBatchToColumnarBatch(input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
conf: SQLConf): RDD[ColumnarBatch] = {
val batches = convertCachedBatchToColumnarInternal(input, cacheAttributes,
selectedAttributes)
val cbRdd = batches.map(batch => {
val cols = GpuColumnVector.extractColumns(batch)
try {
new ColumnarBatch(cols.map(_.copyToHost()).toArray, batch.numRows())
} finally {
cols.map(_.close())
revans2 marked this conversation as resolved.
Show resolved Hide resolved
}
})
cbRdd.mapPartitions(iter => new CloseableColumnBatchIterator(iter))
}

/**
* Convert the cached batch into `InternalRow`s.
* @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.
*/
override def convertCachedBatchToInternalRow(input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
conf: SQLConf): RDD[InternalRow] = {
val requestedColumnIndices = selectedAttributes.map(a =>
cacheAttributes.map(_.exprId).indexOf(a.exprId))
// if plugin enabled
val cbRdd: RDD[InternalRow] = input.map(batch => {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You cannot use a map for this. There is not a one to one mapping between a batch and an internal row. You will have to return an iterator and please look at how GpuColumnarToRowExec does this conversion because it should be much more performant.

Also we need some tests that exercise this code path.

if (batch.isInstanceOf[ParquetCachedBatch]) {
val parquetCB = batch.asInstanceOf[ParquetCachedBatch]
withResource(Table.readParquet(ParquetOptions.DEFAULT, parquetCB.getBuffer(), 0,
revans2 marked this conversation as resolved.
Show resolved Hide resolved
parquetCB.sizeInBytes)) { table =>
withResource(GpuColumnVector.from(table)) { cb =>
val cols = GpuColumnVector.extractColumns(cb)
InternalRow(requestedColumnIndices.map(ordinal => cols(ordinal)).map(_.copyToHost()))
}
}
} else {
throw new IllegalStateException("I don't know how to convert this batch")
}
})
cbRdd
}

/**
* Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data.
* We use the RowToColumnarIterator and convert each batch at a time
* @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.
*/
override def convertInternalRowToCachedBatch(input: RDD[InternalRow],
schema: Seq[Attribute],
storageLevel: StorageLevel,
conf: SQLConf): RDD[CachedBatch] = {
val s = StructType(schema.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
val converters = new GpuRowToColumnConverter(s)
val numInputRows = SQLMetrics.createMetric(input.context, NUM_INPUT_ROWS)
val numOutputBatches = SQLMetrics.createMetric(input.context, NUM_OUTPUT_BATCHES)
val numOutputRows = SQLMetrics.createMetric(input.context, NUM_OUTPUT_ROWS)
val totalTime = SQLMetrics.createMetric(input.context, TOTAL_TIME)
revans2 marked this conversation as resolved.
Show resolved Hide resolved
val columnarBatchRdd = input.mapPartitions(iter => {
new RowToColumnarIterator(iter, s, RequireSingleBatch, converters, totalTime, numInputRows,
numOutputRows, numOutputBatches)
})
columnarBatchRdd.map(cb => {
val cachedBatch = compressColumnarBatchWithParquet(cb)
cb.close()
revans2 marked this conversation as resolved.
Show resolved Hide resolved
cachedBatch
})
}

override def buildFilter(predicates: Seq[Expression],
cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
//essentially a noop
(partId: Int, b: Iterator[CachedBatch]) => b
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,17 +28,20 @@ import org.apache.spark.SparkEnv
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.read.Scan
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec}
import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec
import org.apache.spark.sql.internal.StaticSQLConf
import org.apache.spark.sql.rapids.{GpuFileSourceScanExec, GpuTimeSub, ShuffleManagerShimBase}
import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase
import org.apache.spark.sql.rapids.shims.spark310._
import org.apache.spark.sql.rapids.shims.spark310.{GpuInMemoryTableScanExec, ShuffleManagerShim}
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
import org.apache.spark.unsafe.types.CalendarInterval
Expand Down Expand Up @@ -165,6 +168,21 @@ class Spark310Shims extends Spark301Shims {
canUseSmallFileOpt)
}
}),
GpuOverrides.exec[InMemoryTableScanExec](
"Implementation of InMemoryTableScanExec to use GPU accelerated Caching",
(scan, conf, p, r) => new SparkPlanMeta[InMemoryTableScanExec](scan, conf, p, r) {
override def tagPlanForGpu(): Unit = {
if (!scan.relation.cacheBuilder.serializer.isInstanceOf[ParquetCachedBatchSerializer]) {
willNotWorkOnGpu("DefaultCachedBatchSerializer being used")
revans2 marked this conversation as resolved.
Show resolved Hide resolved
}
}
/**
* Convert InMemoryTableScanExec to a GPU enabled version.
*/
override def convertToGpu(): GpuExec = {
GpuInMemoryTableScanExec(scan.attributes, scan.predicates, scan.relation)
}
}),
GpuOverrides.exec[SortMergeJoinExec](
"Sort merge join, replacing with shuffled hash join",
(join, conf, p, r) => new GpuSortMergeJoinMeta(join, conf, p, r)),
Expand Down Expand Up @@ -247,6 +265,16 @@ class Spark310Shims extends Spark301Shims {

override def copyFileSourceScanExec(scanExec: GpuFileSourceScanExec,
supportsSmallFileOpt: Boolean): GpuFileSourceScanExec = {
scanExec.copy(supportsSmallFileOpt=supportsSmallFileOpt)
scanExec.copy(supportsSmallFileOpt = supportsSmallFileOpt)
}

override def getGpuColumnarToRowTransition(plan: SparkPlan): String = {
val serName = plan.conf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
val serClass = Class.forName(serName)
if (serClass == classOf[ParquetCachedBatchSerializer]) {
classOf[org.apache.spark.sql.rapids.shims.spark310.GpuColumnarToRowExec].getName
revans2 marked this conversation as resolved.
Show resolved Hide resolved
} else {
classOf[GpuColumnarToRowExec].getName
}
}
}
Loading