diff --git a/dist/README.md b/dist/README.md
index 3aa9c583f82..35d530a7653 100644
--- a/dist/README.md
+++ b/dist/README.md
@@ -27,3 +27,4 @@ If you have to change the contents of the uber jar the following files control w
1. `unshimmed-base.txt` - this has classes and files that should go into the base jar with their normal package name (not shaded). This includes user visible classes (ie com/nvidia/spark/SQLPlugin), python files, and other files that aren't version specific. Uses Spark 3.0.1 built jar for these base classes.
2. `unshimmed-extras.txt` - This is applied to all the individual Spark specific verson jars to pull any files that need to go into the base of the jar and not into the Spark specific directory from all of the other Spark version jars.
+3. `unshimmed-spark311.txt` - This is applied to all the Spark 3.1.1 specific verson to pull any files that need to go into the base of the jar and not into the Spark specific directory from all of the other Spark version jars.
diff --git a/dist/pom.xml b/dist/pom.xml
index b35554abf83..835d1187c54 100644
--- a/dist/pom.xml
+++ b/dist/pom.xml
@@ -245,13 +245,19 @@
src="${project.build.directory}/deps/rapids-4-spark-aggregator_${scala.binary.version}-${project.version}-spark311.jar"
dest="${project.build.directory}/parallel-world/"
>
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
-
+
+
+
+
+
Int.MaxValue || newPos < Int.MinValue) {
- throw new IllegalStateException("seek value is out of supported range " + newPos)
- }
- byteBuffer.position(newPos.toInt)
- }
- }
- }
-}
-
-private object ByteArrayOutputFile {
- val BLOCK_SIZE: Int = 32 * 1024 * 1024 // 32M
-}
-
-private class ByteArrayOutputFile(stream: ByteArrayOutputStream) extends OutputFile {
- override def create(blockSizeHint: Long): PositionOutputStream = {
- new DelegatingPositionOutputStream(stream) {
- var pos = 0
-
- override def getPos: Long = pos
-
- override def write(b: Int): Unit = {
- super.write(b)
- pos += Integer.BYTES
- }
-
- override def write(b: Array[Byte]): Unit = {
- super.write(b)
- pos += b.length
- }
-
- override def write(b: Array[Byte], off: Int, len: Int): Unit = {
- super.write(b, off, len)
- pos += len
- }
- }
- }
-
- override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream =
- throw new UnsupportedOperationException("Don't need to overwrite")
-
- override def supportsBlockSize(): Boolean = true
-
- override def defaultBlockSize(): Long = ByteArrayOutputFile.BLOCK_SIZE
-}
-
-private class ParquetBufferConsumer(val numRows: Int) extends HostBufferConsumer with
- AutoCloseable {
- @transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
- private var buffer: Array[Byte] = _
-
- override def handleBuffer(buffer: HostMemoryBuffer, len: Long): Unit = {
- offHeapBuffers += Tuple2(buffer, len)
- }
-
- def getBuffer: Array[Byte] = {
- if (buffer == null) {
- writeBuffers()
- }
- buffer
- }
-
- def close(): Unit = {
- if (buffer == null) {
- writeBuffers()
- }
- }
-
- private def writeBuffers(): Unit = {
- val toProcess = offHeapBuffers.dequeueAll(_ => true)
- // We are making sure the input is smaller than 2gb so the parquet written should never be more
- // than Int.MAX_SIZE.
- val bytes = toProcess.map(_._2).sum
-
- // for now assert bytes are less than Int.MaxValue
- assert(bytes <= Int.MaxValue)
- buffer = new Array(bytes.toInt)
- try {
- var offset: Int = 0
- toProcess.foreach(ops => {
- val origBuffer = ops._1
- val len = ops._2.toInt
- origBuffer.asByteBuffer().get(buffer, offset, len)
- offset = offset + len
- })
- } finally {
- toProcess.map(_._1).safeClose()
- }
- }
-}
-
-private object ParquetCachedBatch {
- def apply(parquetBuff: ParquetBufferConsumer): ParquetCachedBatch = {
- new ParquetCachedBatch(parquetBuff.numRows, parquetBuff.getBuffer)
- }
-}
-
-case class ParquetCachedBatch(
- numRows: Int,
- buffer: Array[Byte]) extends CachedBatch {
- override def sizeInBytes: Long = buffer.length
-}
-
-/**
- * Spark wants the producer to close the batch. We have a listener in this iterator that will close
- * the batch after the task is completed
- */
-private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
- Iterator[ColumnarBatch] {
- var cb: ColumnarBatch = _
-
- private def closeCurrentBatch(): Unit = {
- if (cb != null) {
- cb.close()
- cb = null
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: 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 = schema.fields.forall { f =>
- // only check spark b/c if we are on the GPU then we will be calling the gpu method regardless
- isTypeSupportedByColumnarSparkParquetWriter(f.dataType) || f.dataType == DataTypes.NullType
- }
-
- private def isTypeSupportedByColumnarSparkParquetWriter(dataType: DataType): Boolean = {
- // Columnar writer in Spark only supports AtomicTypes ATM
- dataType match {
- case TimestampType | StringType | BooleanType | DateType | BinaryType |
- DoubleType | FloatType | ByteType | IntegerType | LongType | ShortType => true
- case _: DecimalType => true
- case _ => false
- }
- }
-
- def isSchemaSupportedByCudf(schema: Seq[Attribute]): Boolean = {
- schema.forall(field => isSupportedByCudf(field.dataType))
- }
-
- def isSupportedByCudf(dataType: DataType): Boolean = {
- dataType match {
- // TODO: when arrays are supported for cudf writes add it here.
- // https://github.com/NVIDIA/spark-rapids/issues/2054
- case s: StructType => s.forall(field => isSupportedByCudf(field.dataType))
- case _ => GpuColumnVector.isNonNestedSupportedType(dataType)
- }
- }
-
- /**
- * This method checks if the datatype passed is officially supported by parquet.
- *
- * Please refer to https://github.com/apache/parquet-format/blob/master/LogicalTypes.md to see
- * the what types are supported by parquet
- */
- def isTypeSupportedByParquet(dataType: DataType): Boolean = {
- dataType match {
- case CalendarIntervalType | NullType => false
- case s: StructType => s.forall(field => isTypeSupportedByParquet(field.dataType))
- case ArrayType(elementType, _) => isTypeSupportedByParquet(elementType)
- case MapType(keyType, valueType, _) => isTypeSupportedByParquet(keyType) &&
- isTypeSupportedByParquet(valueType)
- case d: DecimalType if d.scale < 0 => false
- case _ => 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- val structSchema = schemaWithUnambiguousNames.toStructType
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- def putOnGpuIfNeeded(batch: ColumnarBatch): ColumnarBatch = {
- if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
- val s: StructType = structSchema
- val gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows()).build(batch.numRows())
- batch.close()
- gpuCB
- } else {
- batch
- }
- }
-
- input.flatMap(batch => {
- if (batch.numCols() == 0) {
- List(ParquetCachedBatch(batch.numRows(), new Array[Byte](0)))
- } else {
- withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
- compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
- bytesAllowedPerBatch)
- }
- }
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[ColumnarBatch](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getColumnarBatchToCachedBatchIterator
- }
- }
- }
-
- private[rapids] def compressColumnarBatchWithParquet(
- oldGpuCB: ColumnarBatch,
- schema: StructType,
- origSchema: StructType,
- bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
- val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
- oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
- .getBase.getDeviceMemorySize / oldGpuCB.numRows()
- }.sum
-
- val columns = for (i <- 0 until oldGpuCB.numCols()) yield {
- val gpuVector = oldGpuCB.column(i).asInstanceOf[GpuColumnVector]
- var dataType = origSchema(i).dataType
- val v = ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(gpuVector.getBase,
- origSchema(i).dataType,
- // we are checking for scale > 0 because cudf and spark refer to scales as opposites
- // e.g. scale = -3 in Spark is scale = 3 in cudf
- (_, cv) => cv.getType.isDecimalType && cv.getType.getScale > 0,
- (_, cv) => {
- if (cv.getType.isBackedByLong) {
- dataType = LongType
- cv.bitCastTo(DType.INT64)
- } else {
- dataType = IntegerType
- cv.bitCastTo(DType.INT32)
- }
- }
- )
- GpuColumnVector.from(v, schema(i).dataType)
- }
- withResource(new ColumnarBatch(columns.toArray, oldGpuCB.numRows())) { gpuCB =>
- val rowsAllowedInBatch = (bytesAllowedPerBatch / estimatedRowSize).toInt
- val splitIndices = scala.Range(rowsAllowedInBatch, gpuCB.numRows(), rowsAllowedInBatch)
- val buffers = new ListBuffer[ParquetCachedBatch]
- if (splitIndices.nonEmpty) {
- val splitVectors = new ListBuffer[Array[ColumnVector]]
- try {
- for (index <- 0 until gpuCB.numCols()) {
- splitVectors +=
- gpuCB.column(index).asInstanceOf[GpuColumnVector].getBase.split(splitIndices: _*)
- }
-
- // Splitting the table
- // e.g. T0 = {col1, col2,...,coln} => split columns into 'm' cols =>
- // T00= {splitCol1(0), splitCol2(0),...,splitColn(0)}
- // T01= {splitCol1(1), splitCol2(1),...,splitColn(1)}
- // ...
- // T0m= {splitCol1(m), splitCol2(m),...,splitColn(m)}
- def makeTableForIndex(i: Int): Table = {
- val columns = splitVectors.indices.map(j => splitVectors(j)(i))
- new Table(columns: _*)
- }
-
- for (i <- splitVectors.head.indices) {
- withResource(makeTableForIndex(i)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- } finally {
- splitVectors.foreach(array => array.safeClose())
- }
- } else {
- withResource(GpuColumnVector.from(gpuCB)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- buffers.toList
- }
- }
-
- private def writeTableToCachedBatch(
- table: Table,
- schema: StructType): ParquetBufferConsumer = {
- val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
- val opts = GpuParquetFileFormat
- .parquetWriterOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
- .withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
- withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
- writer.write(table)
- }
- buffer
- }
-
- /**
- * This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying 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.
- */
- def gpuConvertCachedBatchToColumnarBatch(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- conf: SQLConf): RDD[ColumnarBatch] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- convertCachedBatchToColumnarInternal(
- input,
- cachedSchemaWithNames,
- selectedSchemaWithNames,
- newSelectedAttributes)
- }
-
- private def convertCachedBatchToColumnarInternal(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- originalSelectedAttributes: Seq[Attribute]): RDD[ColumnarBatch] = {
-
- val cbRdd: RDD[ColumnarBatch] = input.map {
- case parquetCB: ParquetCachedBatch =>
- val parquetOptions = ParquetOptions.builder()
- .includeColumn(selectedAttributes.map(_.name).asJavaCollection).build()
- withResource(Table.readParquet(parquetOptions, parquetCB.buffer, 0,
- parquetCB.sizeInBytes)) { table =>
- withResource {
- for (i <- 0 until table.getNumberOfColumns) yield {
- ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(table.getColumn(i),
- originalSelectedAttributes(i).dataType,
- (dataType, _) => dataType match {
- case d: DecimalType if d.scale < 0 => true
- case _ => false
- },
- (dataType, cv) => {
- //TODO: why do we have to copy to a vector
- dataType match {
- case d: DecimalType =>
- withResource(cv.bitCastTo(DecimalUtil.createCudfDecimal(d))) {
- _.copyToColumnVector()
- }
- case _ =>
- throw new IllegalStateException("We don't cast any type besides Decimal " +
- "with scale < 0")
- }
- }
- )
- }
- } { col =>
- withResource(new Table(col: _*)) { t =>
- GpuColumnVector.from(t, originalSelectedAttributes.map(_.dataType).toArray)
- }
- }
- }
- case _ =>
- throw new IllegalStateException("I don't know how to convert this batch")
- }
- cbRdd
- }
-
- private def getSelectedSchemaFromCachedSchema(
- selectedAttributes: Seq[Attribute],
- cacheAttributes: Seq[Attribute]): Seq[Attribute] = {
- selectedAttributes.map {
- a => cacheAttributes(cacheAttributes.map(_.exprId).indexOf(a.exprId))
- }
- }
-
- /**
- * 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] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val rapidsConf = new RapidsConf(conf)
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- if (rapidsConf.isSqlEnabled &&
- isSchemaSupportedByCudf(cachedSchemaWithNames)) {
- val batches = convertCachedBatchToColumnarInternal(input, cachedSchemaWithNames,
- selectedSchemaWithNames, newSelectedAttributes)
- val cbRdd = batches.map(batch => {
- withResource(batch) { gpuBatch =>
- val cols = GpuColumnVector.extractColumns(gpuBatch)
- new ColumnarBatch(cols.safeMap(_.copyToHost()).toArray, gpuBatch.numRows())
- }
- })
- cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter))
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, newSelectedAttributes, broadcastedConf).getColumnBatchIterator
- }
- }
- }
- }
-
- /**
- * 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 (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes)
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, selectedAttributes, broadcastedConf).getInternalRowIterator
- }
- }
- }
-
- private abstract class UnsupportedDataHandlerIterator extends Iterator[InternalRow] {
-
- def handleInternalRow(schema: Seq[Attribute], row: InternalRow, newRow: InternalRow): Unit
-
- def handleInterval(data: SpecializedGetters, index: Int): Any
-
- def handleStruct(
- data: InternalRow,
- origSchema: StructType,
- supportedSchema: StructType): InternalRow = {
- val structRow = InternalRow.fromSeq(supportedSchema)
- handleInternalRow(origSchema.map(field =>
- AttributeReference(field.name, field.dataType, field.nullable)()), data, structRow)
- structRow
- }
-
- def handleMap(
- keyType: DataType,
- valueType: DataType,
- mapData: MapData): MapData = {
- val keyData = mapData.keyArray()
- val newKeyData = handleArray(keyType, keyData)
- val valueData = mapData.valueArray()
- val newValueData = handleArray(valueType, valueData)
- new ArrayBasedMapData(newKeyData, newValueData)
- }
-
- def handleArray(
- dataType: DataType,
- arrayData: ArrayData): ArrayData = {
- dataType match {
- case s@StructType(_) =>
- val listBuffer = new ListBuffer[InternalRow]()
- val supportedSchema = mapping(dataType).asInstanceOf[StructType]
- arrayData.foreach(supportedSchema, (_, data) => {
- val structRow =
- handleStruct(data.asInstanceOf[InternalRow], s, s)
- listBuffer += structRow.copy()
- })
- new GenericArrayData(listBuffer)
-
- case ArrayType(elementType, _) =>
- val arrayList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val subArrayData = arrayData.getArray(i)
- arrayList.append(handleArray(elementType, subArrayData))
- }
- new GenericArrayData(arrayList)
-
- case m@MapType(_, _, _) =>
- val mapList =
- new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val mapData = arrayData.getMap(i)
- mapList.append(handleMap(m.keyType, m.valueType, mapData))
- }
- new GenericArrayData(mapList)
-
- case CalendarIntervalType =>
- val citList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val citRow = handleInterval(arrayData, i)
- citList += citRow
- }
- new GenericArrayData(citList)
-
- case _ =>
- arrayData
- }
- }
- }
-
- /**
- * Consumes the Iterator[CachedBatch] to return either Iterator[ColumnarBatch] or
- * Iterator[InternalRow]
- */
- private class CachedBatchIteratorConsumer(
- cbIter: Iterator[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- origCacheSchema: Seq[Attribute],
- origRequestedSchema: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val hadoopConf: Configuration = getHadoopConf(origRequestedSchema.toStructType, conf)
- val options: ParquetReadOptions = HadoopReadOptions.builder(hadoopConf).build()
- /**
- * We are getting this method using reflection because its a package-private
- */
- val readBatchMethod: Method =
- classOf[VectorizedColumnReader].getDeclaredMethod("readBatch", Integer.TYPE,
- classOf[WritableColumnVector])
- readBatchMethod.setAccessible(true)
-
- def getInternalRowIterator: Iterator[InternalRow] = {
-
- /**
- * This iterator converts an iterator[CachedBatch] to an iterator[InternalRow].
- *
- * This makes it unlike a regular iterator because CachedBatch => InternalRow* is a 1-n
- * relation. The way we have implemented this is to first go through the
- * iterator[CachedBatch] (cbIter) to look for a valid iterator (iter) i.e. hasNext() => true.
- * Then every time next() is called we return a single InternalRow from iter. When
- * iter.hasNext() => false, we find the next valid iterator in cbIter and the process
- * continues as above.
- */
- new Iterator[InternalRow]() {
-
- var iter: Iterator[InternalRow] = _
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = convertCachedBatchToInternalRowIter
- }
- iter != null && iter.hasNext
- }
-
- override def next(): InternalRow = {
- // will return the next InternalRow if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- /**
- * This method converts a CachedBatch to an iterator of InternalRows.
- */
- private def convertCachedBatchToInternalRowIter: Iterator[InternalRow] = {
- val parquetCachedBatch = cbIter.next().asInstanceOf[ParquetCachedBatch]
- val inputFile = new ByteArrayInputFile(parquetCachedBatch.buffer)
- withResource(ParquetFileReader.open(inputFile, options)) { parquetFileReader =>
- val parquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val hasUnsupportedType = origCacheSchema.exists { field =>
- !isTypeSupportedByParquet(field.dataType)
- }
-
- val unsafeRows = new ArrayBuffer[InternalRow]
- import org.apache.parquet.io.ColumnIOFactory
- var pages = parquetFileReader.readNextRowGroup()
- while (pages != null) {
- val rows = pages.getRowCount
- val columnIO = new ColumnIOFactory().getColumnIO(parquetSchema)
- val recordReader =
- columnIO.getRecordReader(pages, new ParquetRecordMaterializer(parquetSchema,
- cacheAttributes.toStructType,
- new ParquetToSparkSchemaConverter(hadoopConf), None /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED))
- for (_ <- 0 until rows.toInt) {
- val row = recordReader.read
- unsafeRows += row.copy()
- }
- pages = parquetFileReader.readNextRowGroup()
- }
-
- val iter = unsafeRows.iterator
- val unsafeProjection =
- GenerateUnsafeProjection.generate(selectedAttributes, cacheAttributes)
- if (hasUnsupportedType) {
- new UnsupportedDataHandlerIterator() {
- val wrappedIter: Iterator[InternalRow] = iter
- val newRow = new GenericInternalRow(cacheAttributes.length)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- //read a row and convert it to what the caller is expecting
- val row = wrappedIter.next()
- handleInternalRow(origCacheSchema, row, newRow)
- val unsafeProjection =
- GenerateUnsafeProjection.generate(origRequestedSchema, origCacheSchema)
- unsafeProjection.apply(newRow)
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): CalendarInterval = {
- if (data.isNullAt(index)) {
- null
- } else {
- val structData = data.getStruct(index, 3)
- new CalendarInterval(structData.getInt(0),
- structData.getInt(1), structData.getLong(2))
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val supportedSchema = mapping(dataType)
- .asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, supportedSchema.size), s, s)
- newRow.update(index, structRow)
-
- case a@ArrayType(_, _) =>
- val arrayData = row.getArray(index)
- newRow.update(index, handleArray(a.elementType, arrayData))
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- newRow.update(index, handleMap(keyType, valueType, mapData))
-
- case CalendarIntervalType =>
- val interval = handleInterval(row, index)
- if (interval == null) {
- newRow.setNullAt(index)
- } else {
- newRow.setInterval(index, interval)
- }
- case d: DecimalType =>
- if (row.isNullAt(index)) {
- newRow.setDecimal(index, null, d.precision)
- } else {
- val dec = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- Decimal(row.getInt(index).toLong, d.precision, d.scale)
- } else {
- Decimal(row.getLong(index), d.precision, d.scale)
- }
- newRow.update(index, dec)
- }
- case NullType =>
- newRow.setNullAt(index)
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- } else {
- iter.map(unsafeProjection)
- }
- }
- }
- }
- }
-
- private class CurrentBatchIterator(val parquetCachedBatch: ParquetCachedBatch)
- extends Iterator[ColumnarBatch] with AutoCloseable {
-
- val capacity = conf.parquetVectorizedReaderBatchSize
- var columnReaders: Array[VectorizedColumnReader] = _
- val columnVectors: Array[OffHeapColumnVector] =
- OffHeapColumnVector.allocateColumns(capacity, selectedAttributes.toStructType)
- val columnarBatch = new ColumnarBatch(columnVectors
- .asInstanceOf[Array[vectorized.ColumnVector]])
- var rowsReturned: Long = 0L
- var numBatched = 0
- var batchIdx = 0
- var totalCountLoadedSoFar: Long = 0
- val parquetFileReader =
- ParquetFileReader.open(new ByteArrayInputFile(parquetCachedBatch.buffer), options)
- val (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache) = {
- val parquetToSparkSchemaConverter = new ParquetToSparkSchemaConverter(hadoopConf)
- // we are getting parquet schema and then converting it to catalyst schema
- // because catalyst schema that we get from Spark doesn't have the exact schema expected
- // by the columnar parquet reader
- val inMemCacheParquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val inMemCacheSparkSchema = parquetToSparkSchemaConverter.convert(inMemCacheParquetSchema)
-
- val totalRowCount = parquetFileReader.getRowGroups.asScala.map(_.getRowCount).sum
- val sparkToParquetSchemaConverter = new SparkToParquetSchemaConverter(hadoopConf)
- val inMemReqSparkSchema = StructType(selectedAttributes.toStructType.map { field =>
- inMemCacheSparkSchema.fields(inMemCacheSparkSchema.fieldIndex(field.name))
- })
- val inMemReqParquetSchema = sparkToParquetSchemaConverter.convert(inMemReqSparkSchema)
- val columnsRequested: util.List[ColumnDescriptor] = inMemReqParquetSchema.getColumns
-
- val reqSparkSchemaInCacheOrder = StructType(inMemCacheSparkSchema.filter(f =>
- inMemReqSparkSchema.fields.exists(f0 => f0.name.equals(f.name))))
-
- // There could be a case especially in a distributed environment where the requestedSchema
- // and cacheSchema are not in the same order. We need to create a map so we can guarantee
- // that we writing to the correct columnVector
- val cacheSchemaToReqSchemaMap: Map[Int, Int] =
- reqSparkSchemaInCacheOrder.indices.map { index =>
- index -> inMemReqSparkSchema.fields.indexOf(reqSparkSchemaInCacheOrder.fields(index))
- }.toMap
-
- val reqParquetSchemaInCacheOrder =
- sparkToParquetSchemaConverter.convert(reqSparkSchemaInCacheOrder)
-
- // reset spark schema calculated from parquet schema
- hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, inMemReqSparkSchema.json)
- hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, inMemReqSparkSchema.json)
-
- val columnsInCache: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
- val typesInCache: util.List[Type] = reqParquetSchemaInCacheOrder.asGroupType.getFields
- val missingColumns = new Array[Boolean](inMemReqParquetSchema.getFieldCount)
-
- // initialize missingColumns to cover the case where requested column isn't present in the
- // cache, which should never happen but just in case it does
- val paths: util.List[Array[String]] = inMemReqParquetSchema.getPaths
-
- for (i <- 0 until inMemReqParquetSchema.getFieldCount) {
- val t = inMemReqParquetSchema.getFields.get(i)
- if (!t.isPrimitive || t.isRepetition(Type.Repetition.REPEATED)) {
- throw new UnsupportedOperationException("Complex types not supported.")
- }
- val colPath = paths.get(i)
- if (inMemCacheParquetSchema.containsPath(colPath)) {
- val fd = inMemCacheParquetSchema.getColumnDescription(colPath)
- if (!(fd == columnsRequested.get(i))) {
- throw new UnsupportedOperationException("Schema evolution not supported.")
- }
- missingColumns(i) = false
- } else {
- if (columnsRequested.get(i).getMaxDefinitionLevel == 0) {
- // Column is missing in data but the required data is non-nullable.
- // This file is invalid.
- throw new IOException(s"Required column is missing in data file: ${colPath.toList}")
- }
- missingColumns(i) = true
- }
- }
-
- for (i <- missingColumns.indices) {
- if (missingColumns(i)) {
- columnVectors(i).putNulls(0, capacity)
- columnVectors(i).setIsConstant()
- }
- }
-
- (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache)
- }
-
- @throws[IOException]
- def checkEndOfRowGroup(): Unit = {
- if (rowsReturned != totalCountLoadedSoFar) return
- val pages = parquetFileReader.readNextRowGroup
- if (pages == null) {
- throw new IOException("expecting more rows but reached last" +
- " block. Read " + rowsReturned + " out of " + totalRowCount)
- }
- columnReaders = new Array[VectorizedColumnReader](columnsRequested.size)
- for (i <- 0 until columnsRequested.size) {
- if (!missingColumns(i)) {
- columnReaders(i) =
- new VectorizedColumnReader(
- columnsInCache.get(i),
- typesInCache.get(i).getOriginalType,
- pages.getPageReader(columnsInCache.get(i)),
- null /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED.toString,
- LegacyBehaviorPolicy.EXCEPTION.toString)
- }
- }
- totalCountLoadedSoFar += pages.getRowCount
- }
-
- /**
- * Read the next RowGroup and read each column and return the columnarBatch
- */
- def nextBatch: Boolean = {
- for (vector <- columnVectors) {
- vector.reset()
- }
- columnarBatch.setNumRows(0)
- if (rowsReturned >= totalRowCount) return false
- checkEndOfRowGroup()
- val num = Math.min(capacity.toLong, totalCountLoadedSoFar - rowsReturned).toInt
- for (i <- columnReaders.indices) {
- if (columnReaders(i) != null) {
- readBatchMethod.invoke(columnReaders(i), num.asInstanceOf[AnyRef],
- columnVectors(cacheSchemaToReqSchemaMap(i)).asInstanceOf[AnyRef])
- }
- }
- rowsReturned += num
- columnarBatch.setNumRows(num)
- numBatched = num
- batchIdx = 0
- true
- }
-
- override def hasNext: Boolean = rowsReturned < totalRowCount
-
- override def next(): ColumnarBatch = {
- if (nextBatch) {
- // FYI, A very IMPORTANT thing to note is that we are returning the columnar batch
- // as-is i.e. this batch has NullTypes saved as IntegerTypes with null values. The
- // way Spark optimizes the read of NullTypes makes this work without having to rip out
- // the IntegerType column to be replaced by a NullType column. This could change in
- // future and will affect this code.
- columnarBatch
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => {
- close()
- })
-
- override def close(): Unit = {
- parquetFileReader.close()
- }
- }
-
- /**
- * This method returns a ColumnarBatch iterator over a CachedBatch.
- * Each CachedBatch => ColumnarBatch is a 1-1 conversion so its pretty straight forward
- */
- def getColumnBatchIterator: Iterator[ColumnarBatch] = new Iterator[ColumnarBatch] {
- var iter = getIterator
-
- def getIterator: Iterator[ColumnarBatch] = {
- if (!cbIter.hasNext) {
- Iterator.empty
- } else {
- new CurrentBatchIterator(cbIter.next().asInstanceOf[ParquetCachedBatch])
- }
- }
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = getIterator
- }
- iter != null && iter.hasNext
- }
-
- override def next(): ColumnarBatch = {
- // will return the next ColumnarBatch if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
- }
- }
-
- private def getConfFromMap(sharedConf: Broadcast[Map[String, String]]): SQLConf = {
- val conf = new SQLConf()
- sharedConf.value.foreach { case (k, v) => conf.setConfString(k, v) }
- conf
- }
-
- private val intervalStructType = new StructType()
- .add("_days", IntegerType)
- .add("_months", IntegerType)
- .add("_ms", LongType)
-
- def getBytesAllowedPerBatch(conf: SQLConf): Long = {
- val gpuBatchSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf)
- // we are rough estimating 0.5% as meta_data_size. we can do better estimation in future
- val approxMetaDataSizeBytes = gpuBatchSize * 0.5/100
- (gpuBatchSize - approxMetaDataSizeBytes).toLong
- }
-
- /**
- * This is a private helper class to return Iterator to convert InternalRow or ColumnarBatch to
- * CachedBatch. There is no type checking so if the type of T is anything besides InternalRow
- * or ColumnarBatch then the behavior is undefined.
- *
- * @param iter - an iterator over InternalRow or ColumnarBatch
- * @param cachedAttributes - Schema of the cached batch
- * @param sharedConf - SQL conf
- * @tparam T - Strictly either InternalRow or ColumnarBatch
- */
- private[rapids] class CachedBatchIteratorProducer[T](
- iter: Iterator[T],
- cachedAttributes: Seq[Attribute],
- origCachedAttributes: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val hadoopConf: Configuration = getHadoopConf(cachedAttributes.toStructType, conf)
-
- def getInternalRowToCachedBatchIterator: Iterator[CachedBatch] = {
- new InternalRowToCachedBatchIterator
- }
-
- def getColumnarBatchToCachedBatchIterator: Iterator[CachedBatch] = {
- new ColumnarBatchToCachedBatchIterator
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[InternalRow]. This is a n-1
- * relationship. Each partition represents a single parquet file, so we encode it
- * and return the CachedBatch when next is called.
- */
- class InternalRowToCachedBatchIterator extends Iterator[CachedBatch]() {
-
- var parquetOutputFileFormat = new ParquetOutputFileFormat()
-
- // For testing only
- private[rapids] def setParquetOutputFileFormat(p: ParquetOutputFileFormat): Unit = {
- parquetOutputFileFormat = p
- }
-
- // is there a type that spark doesn't support by default in the schema?
- val hasUnsupportedType: Boolean = origCachedAttributes.exists { attribute =>
- !isTypeSupportedByParquet(attribute.dataType)
- }
-
- def getIterator: Iterator[InternalRow] = {
- if (!hasUnsupportedType) {
- iter.asInstanceOf[Iterator[InternalRow]]
- } else {
- new UnsupportedDataHandlerIterator {
-
- val wrappedIter: Iterator[InternalRow] = iter.asInstanceOf[Iterator[InternalRow]]
-
- val newRow: InternalRow = InternalRow.fromSeq(cachedAttributes)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- val row = wrappedIter.next()
- handleInternalRow(origCachedAttributes, row, newRow)
- newRow
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): InternalRow = {
- val citRow = InternalRow(IntegerType, IntegerType, LongType)
- if (data.isNullAt(index)) {
- null
- } else {
- val cit = data.getInterval(index)
- citRow.setInt(0, cit.months)
- citRow.setInt(1, cit.days)
- citRow.setLong(2, cit.microseconds)
- citRow
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val newSchema = mapping(dataType).asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, s.fields.length), s, newSchema)
- newRow.update(index, structRow)
-
- case ArrayType(arrayDataType, _) =>
- val arrayData = row.getArray(index)
- val newArrayData = handleArray(arrayDataType, arrayData)
- newRow.update(index, newArrayData)
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- val map = handleMap(keyType, valueType, mapData)
- newRow.update(index, map)
-
- case CalendarIntervalType =>
- val structData: InternalRow = handleInterval(row, index)
- if (structData == null) {
- newRow.setNullAt(index)
- } else {
- newRow.update(index, structData)
- }
-
- case d: DecimalType if d.scale < 0 =>
- if (d.precision <= Decimal.MAX_INT_DIGITS) {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong.toInt)
- } else {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong)
- }
-
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- }
- }
-
- override def hasNext: Boolean = queue.nonEmpty || iter.hasNext
-
- private val queue = new mutable.Queue[CachedBatch]()
-
- //estimate the size of a row
- val estimatedSize: Int = cachedAttributes.map { attr =>
- attr.dataType.defaultSize
- }.sum
-
- override def next(): CachedBatch = {
- if (queue.isEmpty) {
- // to store a row if we have read it but there is no room in the parquet file to put it
- // we will put it in the next CachedBatch
- var leftOverRow: Option[InternalRow] = None
- val rowIterator = getIterator
- while (rowIterator.hasNext || leftOverRow.nonEmpty) {
- // Each partition will be a single parquet file
- var rows = 0
- // at least a single block
- val stream = new ByteArrayOutputStream(ByteArrayOutputFile.BLOCK_SIZE)
- val outputFile: OutputFile = new ByteArrayOutputFile(stream)
- conf.setConfString(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
- val recordWriter = SQLConf.withExistingConf(conf) {
- parquetOutputFileFormat.getRecordWriter(outputFile, hadoopConf)
- }
- var totalSize = 0
- while ((rowIterator.hasNext || leftOverRow.nonEmpty)
- && totalSize < bytesAllowedPerBatch) {
-
- val row = if (leftOverRow.nonEmpty) {
- val a = leftOverRow.get
- leftOverRow = None // reset value
- a
- } else {
- rowIterator.next()
- }
- totalSize += {
- row match {
- case r: UnsafeRow =>
- r.getSizeInBytes
- case _ =>
- estimatedSize
- }
- }
- if (totalSize <= bytesAllowedPerBatch) {
- rows += 1
- if (rows < 0) {
- throw new IllegalStateException("CachedBatch doesn't support rows larger " +
- "than Int.MaxValue")
- }
- recordWriter.write(null, row)
- } else {
- leftOverRow = Some(if (row.isInstanceOf[UnsafeRow]) {
- row.copy()
- } else {
- row
- })
- }
- }
- // passing null as context isn't used in this method
- recordWriter.close(null)
- queue += ParquetCachedBatch(rows, stream.toByteArray)
- }
- }
- queue.dequeue()
- }
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[ColumnarBatch]. This is a 1-1
- * relationship. Each ColumnarBatch is converted to a single ParquetCachedBatch when next()
- * is called on this iterator
- */
- class ColumnarBatchToCachedBatchIterator extends InternalRowToCachedBatchIterator {
- override def getIterator: Iterator[InternalRow] = {
-
- new Iterator[InternalRow] {
- // We have to check for null context because of the unit test
- Option(TaskContext.get).foreach(_.addTaskCompletionListener[Unit](_ => hostBatch.close()))
-
- val batch: ColumnarBatch = iter.asInstanceOf[Iterator[ColumnarBatch]].next
- val hostBatch = if (batch.column(0).isInstanceOf[GpuColumnVector]) {
- withResource(batch) { batch =>
- new ColumnarBatch(batch.safeMap(_.copyToHost()).toArray, batch.numRows())
- }
- } else {
- batch
- }
-
- val rowIterator = hostBatch.rowIterator().asScala
-
- override def next: InternalRow = rowIterator.next
-
- override def hasNext: Boolean = rowIterator.hasNext
-
- }
- }
- }
-
- }
-
- val mapping = new mutable.HashMap[DataType, DataType]()
-
- def getSupportedDataType(curId: AtomicLong, dataType: DataType): DataType = {
- dataType match {
- case CalendarIntervalType =>
- intervalStructType
- case NullType =>
- ByteType
- case s: StructType =>
- val newStructType = StructType(
- s.indices.map { index =>
- StructField(curId.getAndIncrement().toString,
- getSupportedDataType(curId, s.fields(index).dataType), s.fields(index).nullable,
- s.fields(index).metadata)
- })
- mapping.put(s, newStructType)
- newStructType
- case a@ArrayType(elementType, nullable) =>
- val newArrayType =
- ArrayType(getSupportedDataType(curId, elementType), nullable)
- mapping.put(a, newArrayType)
- newArrayType
- case m@MapType(keyType, valueType, nullable) =>
- val newKeyType = getSupportedDataType(curId, keyType)
- val newValueType = getSupportedDataType(curId, valueType)
- val mapType = MapType(newKeyType, newValueType, nullable)
- mapping.put(m, mapType)
- mapType
- case d: DecimalType if d.scale < 0 =>
- val newType = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- IntegerType
- } else {
- LongType
- }
- newType
- case _ =>
- dataType
- }
- }
-
- private def getSupportedSchemaFromUnsupported(
- cachedAttributes: Seq[Attribute],
- requestedAttributes: Seq[Attribute] = Seq.empty): (Seq[Attribute], Seq[Attribute]) = {
-
- // We only handle CalendarIntervalType, Decimals and NullType ATM convert it to a supported type
- val curId = new AtomicLong()
- val newCachedAttributes = cachedAttributes.map {
- attribute => val name = s"_col${curId.getAndIncrement()}"
- attribute.dataType match {
- case CalendarIntervalType =>
- AttributeReference(name, intervalStructType,
- attribute.nullable, metadata = attribute.metadata)(attribute.exprId)
- .asInstanceOf[Attribute]
- case NullType =>
- AttributeReference(name, DataTypes.ByteType,
- nullable = true, metadata =
- attribute.metadata)(attribute.exprId).asInstanceOf[Attribute]
- case StructType(_) | ArrayType(_, _) | MapType(_, _, _) | DecimalType() =>
- AttributeReference(name,
- getSupportedDataType(curId, attribute.dataType),
- attribute.nullable, attribute.metadata)(attribute.exprId)
- case _ =>
- attribute.withName(name)
- }
- }
-
- val newRequestedAttributes =
- getSelectedSchemaFromCachedSchema(requestedAttributes, newCachedAttributes)
-
- (newCachedAttributes, newRequestedAttributes)
- }
-
- private def getHadoopConf(requestedSchema: StructType,
- sqlConf: SQLConf): Configuration = {
-
- val hadoopConf = new Configuration(false)
- hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
- hadoopConf.set(
- ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- ParquetWriteSupport.SPARK_ROW_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sqlConf.sessionLocalTimeZone)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, false)
- hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, false)
-
- hadoopConf.set(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
-
- hadoopConf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key,
- SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, false)
-
- hadoopConf.set(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString)
-
- ParquetWriteSupport.setSchema(requestedSchema, hadoopConf)
-
- hadoopConf
- }
-
- /**
- * 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- val structSchema = schemaWithUnambiguousNames.toStructType
- val converters = new GpuRowToColumnConverter(structSchema)
- val columnarBatchRdd = input.mapPartitions(iter => {
- new RowToColumnarIterator(iter, structSchema, RequireSingleBatch, converters)
- })
- columnarBatchRdd.flatMap(cb => {
- withResource(cb)(cb => compressColumnarBatchWithParquet(cb, structSchema,
- schema.toStructType, bytesAllowedPerBatch))
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[InternalRow](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getInternalRowToCachedBatchIterator
- }
- }
- }
-
- override def buildFilter(
- predicates: Seq[Expression],
- cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
- //essentially a noop
- (_: Int, b: Iterator[CachedBatch]) => b
- }
-}
-
-/**
- * Similar to ParquetFileFormat
- */
-private[rapids] class ParquetOutputFileFormat {
-
- def getRecordWriter(output: OutputFile, conf: Configuration): RecordWriter[Void, InternalRow] = {
- import ParquetOutputFormat._
-
- val blockSize = getLongBlockSize(conf)
- val maxPaddingSize =
- conf.getInt(MAX_PADDING_BYTES, ParquetWriter.MAX_PADDING_SIZE_DEFAULT)
- val validating = getValidation(conf)
-
- val writeSupport = new ParquetWriteSupport().asInstanceOf[WriteSupport[InternalRow]]
- val init = writeSupport.init(conf)
- val writer = new ParquetFileWriter(output, init.getSchema,
- Mode.CREATE, blockSize, maxPaddingSize)
- writer.start()
-
- val writerVersion =
- ParquetProperties.WriterVersion.fromString(conf.get(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString))
-
- val codecFactory = new CodecFactory(conf, getPageSize(conf))
-
- new ParquetRecordWriter[InternalRow](writer, writeSupport, init.getSchema,
- init.getExtraMetaData, blockSize, getPageSize(conf),
- codecFactory.getCompressor(CompressionCodecName.UNCOMPRESSED), getDictionaryPageSize(conf),
- getEnableDictionary(conf), validating, writerVersion,
- ParquetOutputFileFormat.getMemoryManager(conf))
-
- }
-}
-
-private object ParquetOutputFileFormat {
- var memoryManager: MemoryManager = _
- val DEFAULT_MEMORY_POOL_RATIO: Float = 0.95f
- val DEFAULT_MIN_MEMORY_ALLOCATION: Long = 1 * 1024 * 1024 // 1MB
-
- def getMemoryManager(conf: Configuration): MemoryManager = {
- synchronized {
- if (memoryManager == null) {
- import ParquetOutputFormat._
- val maxLoad = conf.getFloat(MEMORY_POOL_RATIO, DEFAULT_MEMORY_POOL_RATIO)
- val minAllocation = conf.getLong(MIN_MEMORY_ALLOCATION, DEFAULT_MIN_MEMORY_ALLOCATION)
- memoryManager = new MemoryManager(maxLoad, minAllocation)
- }
- }
- memoryManager
- }
-}
diff --git a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/SparkBaseShims.scala b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/SparkBaseShims.scala
index c770eed88c8..ca8753f254f 100644
--- a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/SparkBaseShims.scala
+++ b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/SparkBaseShims.scala
@@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.shims.spark311
import java.net.URI
import java.nio.ByteBuffer
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.v2.Spark30XShims
import org.apache.arrow.memory.ReferenceManager
@@ -58,6 +59,7 @@ import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuB
import org.apache.spark.sql.rapids.execution.python.GpuPythonUDF
import org.apache.spark.sql.rapids.execution.python.shims.spark311._
import org.apache.spark.sql.rapids.shims.spark311._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
diff --git a/shims/spark311/src/main/scala/org/apache/spark/sql/rapids/shims/spark311/GpuInMemoryTableScanExec.scala b/shims/spark311/src/main/scala/org/apache/spark/sql/rapids/shims/spark311/GpuInMemoryTableScanExec.scala
deleted file mode 100644
index d180fa7c4e4..00000000000
--- a/shims/spark311/src/main/scala/org/apache/spark/sql/rapids/shims/spark311/GpuInMemoryTableScanExec.scala
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright (c) 2019-2021, NVIDIA CORPORATION.
- *
- * 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 org.apache.spark.sql.rapids.shims.spark311
-
-import com.nvidia.spark.rapids.{GpuExec, GpuMetric}
-import com.nvidia.spark.rapids.shims.spark311.ParquetCachedBatchSerializer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.physical.Partitioning
-import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
-import org.apache.spark.sql.vectorized.ColumnarBatch
-
-case class GpuInMemoryTableScanExec(
- attributes: Seq[Attribute],
- predicates: Seq[Expression],
- @transient relation: InMemoryRelation) extends LeafExecNode with GpuExec {
-
- override val nodeName: String = {
- relation.cacheBuilder.tableName match {
- case Some(_) =>
- "Scan " + relation.cacheBuilder.cachedName
- case _ =>
- super.nodeName
- }
- }
-
- override def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren
-
- override def doCanonicalize(): SparkPlan =
- copy(attributes = attributes.map(QueryPlan.normalizeExpressions(_, relation.output)),
- predicates = predicates.map(QueryPlan.normalizeExpressions(_, relation.output)),
- relation = relation.canonicalized.asInstanceOf[InMemoryRelation])
-
- override def vectorTypes: Option[Seq[String]] =
- relation.cacheBuilder.serializer.vectorTypes(attributes, conf)
-
- private lazy val columnarInputRDD: RDD[ColumnarBatch] = {
- val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS)
- val buffers = filteredCachedBatches()
- relation.cacheBuilder.serializer.asInstanceOf[ParquetCachedBatchSerializer]
- .gpuConvertCachedBatchToColumnarBatch(
- buffers,
- relation.output,
- attributes,
- conf).map { cb =>
- numOutputRows += cb.numRows()
- cb
- }
- }
-
- override def output: Seq[Attribute] = attributes
-
- private def updateAttribute(expr: Expression): Expression = {
- // attributes can be pruned so using relation's output.
- // E.g., relation.output is [id, item] but this scan's output can be [item] only.
- val attrMap = AttributeMap(relation.cachedPlan.output.zip(relation.output))
- expr.transform {
- case attr: Attribute => attrMap.getOrElse(attr, attr)
- }
- }
-
- // The cached version does not change the outputPartitioning of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputPartitioning: Partitioning = {
- relation.cachedPlan.outputPartitioning match {
- case e: Expression => updateAttribute(e).asInstanceOf[Partitioning]
- case other => other
- }
- }
-
- // The cached version does not change the outputOrdering of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputOrdering: Seq[SortOrder] =
- relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder])
-
- lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled
-
- // Accumulators used for testing purposes
- lazy val readPartitions = sparkContext.longAccumulator
- lazy val readBatches = sparkContext.longAccumulator
-
- private def filteredCachedBatches() = {
- // Right now just return the batch without filtering
- relation.cacheBuilder.cachedColumnBuffers
- }
-
- protected override def doExecute(): RDD[InternalRow] = {
- throw new UnsupportedOperationException("This Exec only deals with Columnar Data")
- }
-
- protected override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- columnarInputRDD
- }
-}
diff --git a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/ParquetCachedBatchSerializer.scala b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/ParquetCachedBatchSerializer.scala
deleted file mode 100644
index a31fcb72556..00000000000
--- a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/ParquetCachedBatchSerializer.scala
+++ /dev/null
@@ -1,1525 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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.spark311cdh
-
-import java.io.{InputStream, IOException}
-import java.lang.reflect.Method
-import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-
-import ai.rapids.cudf._
-import ai.rapids.cudf.ParquetWriterOptions.StatisticsFrequency
-import com.nvidia.spark.rapids._
-import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
-import com.nvidia.spark.rapids.RapidsPluginImplicits._
-import java.util
-import org.apache.commons.io.output.ByteArrayOutputStream
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.RecordWriter
-import org.apache.parquet.{HadoopReadOptions, ParquetReadOptions}
-import org.apache.parquet.column.{ColumnDescriptor, ParquetProperties}
-import org.apache.parquet.hadoop.{CodecFactory, MemoryManager, ParquetFileReader, ParquetFileWriter, ParquetInputFormat, ParquetOutputFormat, ParquetRecordWriter, ParquetWriter}
-import org.apache.parquet.hadoop.ParquetFileWriter.Mode
-import org.apache.parquet.hadoop.api.WriteSupport
-import org.apache.parquet.hadoop.metadata.CompressionCodecName
-import org.apache.parquet.io.{DelegatingPositionOutputStream, DelegatingSeekableInputStream, InputFile, OutputFile, PositionOutputStream, SeekableInputStream}
-import org.apache.parquet.schema.{MessageType, Type}
-
-import org.apache.spark.TaskContext
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{vectorized, SparkSession}
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericInternalRow, SpecializedGetters, UnsafeRow}
-import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
-import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
-import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer}
-import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetToSparkSchemaConverter, ParquetWriteSupport, SparkToParquetSchemaConverter, VectorizedColumnReader}
-import org.apache.spark.sql.execution.datasources.parquet.rapids.shims.spark311cdh.ParquetRecordMaterializer
-import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.unsafe.types.CalendarInterval
-
-/**
- * copied from Spark org.apache.spark.util.ByteBufferInputStream
- */
-private class ByteBufferInputStream(private var buffer: ByteBuffer)
- extends InputStream {
-
- override def read(): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- buffer.get() & 0xFF
- }
- }
-
- override def read(dest: Array[Byte]): Int = {
- read(dest, 0, dest.length)
- }
-
- override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- val amountToGet = math.min(buffer.remaining(), length)
- buffer.get(dest, offset, amountToGet)
- amountToGet
- }
- }
-
- override def skip(bytes: Long): Long = {
- if (buffer != null) {
- val amountToSkip = math.min(bytes, buffer.remaining).toInt
- buffer.position(buffer.position() + amountToSkip)
- if (buffer.remaining() == 0) {
- cleanUp()
- }
- amountToSkip
- } else {
- 0L
- }
- }
-
- /**
- * Clean up the buffer, and potentially dispose of it using StorageUtils.dispose().
- */
- private def cleanUp(): Unit = {
- if (buffer != null) {
- buffer = null
- }
- }
-}
-
-class ByteArrayInputFile(buff: Array[Byte]) extends InputFile {
-
- override def getLength: Long = buff.length
-
- override def newStream(): SeekableInputStream = {
- val byteBuffer = ByteBuffer.wrap(buff)
- new DelegatingSeekableInputStream(new ByteBufferInputStream(byteBuffer)) {
- override def getPos: Long = byteBuffer.position()
-
- override def seek(newPos: Long): Unit = {
- if (newPos > Int.MaxValue || newPos < Int.MinValue) {
- throw new IllegalStateException("seek value is out of supported range " + newPos)
- }
- byteBuffer.position(newPos.toInt)
- }
- }
- }
-}
-
-private object ByteArrayOutputFile {
- val BLOCK_SIZE: Int = 32 * 1024 * 1024 // 32M
-}
-
-private class ByteArrayOutputFile(stream: ByteArrayOutputStream) extends OutputFile {
- override def create(blockSizeHint: Long): PositionOutputStream = {
- new DelegatingPositionOutputStream(stream) {
- var pos = 0
-
- override def getPos: Long = pos
-
- override def write(b: Int): Unit = {
- super.write(b)
- pos += Integer.BYTES
- }
-
- override def write(b: Array[Byte]): Unit = {
- super.write(b)
- pos += b.length
- }
-
- override def write(b: Array[Byte], off: Int, len: Int): Unit = {
- super.write(b, off, len)
- pos += len
- }
- }
- }
-
- override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream =
- throw new UnsupportedOperationException("Don't need to overwrite")
-
- override def supportsBlockSize(): Boolean = true
-
- override def defaultBlockSize(): Long = ByteArrayOutputFile.BLOCK_SIZE
-}
-
-private class ParquetBufferConsumer(val numRows: Int) extends HostBufferConsumer with
- AutoCloseable {
- @transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
- private var buffer: Array[Byte] = _
-
- override def handleBuffer(buffer: HostMemoryBuffer, len: Long): Unit = {
- offHeapBuffers += Tuple2(buffer, len)
- }
-
- def getBuffer: Array[Byte] = {
- if (buffer == null) {
- writeBuffers()
- }
- buffer
- }
-
- def close(): Unit = {
- if (buffer == null) {
- writeBuffers()
- }
- }
-
- private def writeBuffers(): Unit = {
- val toProcess = offHeapBuffers.dequeueAll(_ => true)
- // We are making sure the input is smaller than 2gb so the parquet written should never be more
- // than Int.MAX_SIZE.
- val bytes = toProcess.map(_._2).sum
-
- // for now assert bytes are less than Int.MaxValue
- assert(bytes <= Int.MaxValue)
- buffer = new Array(bytes.toInt)
- try {
- var offset: Int = 0
- toProcess.foreach(ops => {
- val origBuffer = ops._1
- val len = ops._2.toInt
- origBuffer.asByteBuffer().get(buffer, offset, len)
- offset = offset + len
- })
- } finally {
- toProcess.map(_._1).safeClose()
- }
- }
-}
-
-private object ParquetCachedBatch {
- def apply(parquetBuff: ParquetBufferConsumer): ParquetCachedBatch = {
- new ParquetCachedBatch(parquetBuff.numRows, parquetBuff.getBuffer)
- }
-}
-
-case class ParquetCachedBatch(
- numRows: Int,
- buffer: Array[Byte]) extends CachedBatch {
- override def sizeInBytes: Long = buffer.length
-}
-
-/**
- * Spark wants the producer to close the batch. We have a listener in this iterator that will close
- * the batch after the task is completed
- */
-private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
- Iterator[ColumnarBatch] {
- var cb: ColumnarBatch = _
-
- private def closeCurrentBatch(): Unit = {
- if (cb != null) {
- cb.close()
- cb = null
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: 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 = schema.fields.forall { f =>
- // only check spark b/c if we are on the GPU then we will be calling the gpu method regardless
- isTypeSupportedByColumnarSparkParquetWriter(f.dataType) || f.dataType == DataTypes.NullType
- }
-
- private def isTypeSupportedByColumnarSparkParquetWriter(dataType: DataType): Boolean = {
- // Columnar writer in Spark only supports AtomicTypes ATM
- dataType match {
- case TimestampType | StringType | BooleanType | DateType | BinaryType |
- DoubleType | FloatType | ByteType | IntegerType | LongType | ShortType => true
- case _: DecimalType => true
- case _ => false
- }
- }
-
- def isSchemaSupportedByCudf(schema: Seq[Attribute]): Boolean = {
- schema.forall(field => isSupportedByCudf(field.dataType))
- }
-
- def isSupportedByCudf(dataType: DataType): Boolean = {
- dataType match {
- // TODO: when arrays are supported for cudf writes add it here.
- // https://github.com/NVIDIA/spark-rapids/issues/2054
- case s: StructType => s.forall(field => isSupportedByCudf(field.dataType))
- case _ => GpuColumnVector.isNonNestedSupportedType(dataType)
- }
- }
-
- /**
- * This method checks if the datatype passed is officially supported by parquet.
- *
- * Please refer to https://github.com/apache/parquet-format/blob/master/LogicalTypes.md to see
- * the what types are supported by parquet
- */
- def isTypeSupportedByParquet(dataType: DataType): Boolean = {
- dataType match {
- case CalendarIntervalType | NullType => false
- case s: StructType => s.forall(field => isTypeSupportedByParquet(field.dataType))
- case ArrayType(elementType, _) => isTypeSupportedByParquet(elementType)
- case MapType(keyType, valueType, _) => isTypeSupportedByParquet(keyType) &&
- isTypeSupportedByParquet(valueType)
- case d: DecimalType if d.scale < 0 => false
- case _ => 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- val structSchema = schemaWithUnambiguousNames.toStructType
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- def putOnGpuIfNeeded(batch: ColumnarBatch): ColumnarBatch = {
- if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
- val s: StructType = structSchema
- val gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows()).build(batch.numRows())
- batch.close()
- gpuCB
- } else {
- batch
- }
- }
-
- input.flatMap(batch => {
- if (batch.numCols() == 0) {
- List(ParquetCachedBatch(batch.numRows(), new Array[Byte](0)))
- } else {
- withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
- compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
- bytesAllowedPerBatch)
- }
- }
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[ColumnarBatch](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getColumnarBatchToCachedBatchIterator
- }
- }
- }
-
- private[rapids] def compressColumnarBatchWithParquet(
- oldGpuCB: ColumnarBatch,
- schema: StructType,
- origSchema: StructType,
- bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
- val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
- oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
- .getBase.getDeviceMemorySize / oldGpuCB.numRows()
- }.sum
-
- val columns = for (i <- 0 until oldGpuCB.numCols()) yield {
- val gpuVector = oldGpuCB.column(i).asInstanceOf[GpuColumnVector]
- var dataType = origSchema(i).dataType
- val v = ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(gpuVector.getBase,
- origSchema(i).dataType,
- // we are checking for scale > 0 because cudf and spark refer to scales as opposites
- // e.g. scale = -3 in Spark is scale = 3 in cudf
- (_, cv) => cv.getType.isDecimalType && cv.getType.getScale > 0,
- (_, cv) => {
- if (cv.getType.isBackedByLong) {
- dataType = LongType
- cv.bitCastTo(DType.INT64)
- } else {
- dataType = IntegerType
- cv.bitCastTo(DType.INT32)
- }
- }
- )
- GpuColumnVector.from(v, schema(i).dataType)
- }
- withResource(new ColumnarBatch(columns.toArray, oldGpuCB.numRows())) { gpuCB =>
- val rowsAllowedInBatch = (bytesAllowedPerBatch / estimatedRowSize).toInt
- val splitIndices = scala.Range(rowsAllowedInBatch, gpuCB.numRows(), rowsAllowedInBatch)
- val buffers = new ListBuffer[ParquetCachedBatch]
- if (splitIndices.nonEmpty) {
- val splitVectors = new ListBuffer[Array[ColumnVector]]
- try {
- for (index <- 0 until gpuCB.numCols()) {
- splitVectors +=
- gpuCB.column(index).asInstanceOf[GpuColumnVector].getBase.split(splitIndices: _*)
- }
-
- // Splitting the table
- // e.g. T0 = {col1, col2,...,coln} => split columns into 'm' cols =>
- // T00= {splitCol1(0), splitCol2(0),...,splitColn(0)}
- // T01= {splitCol1(1), splitCol2(1),...,splitColn(1)}
- // ...
- // T0m= {splitCol1(m), splitCol2(m),...,splitColn(m)}
- def makeTableForIndex(i: Int): Table = {
- val columns = splitVectors.indices.map(j => splitVectors(j)(i))
- new Table(columns: _*)
- }
-
- for (i <- splitVectors.head.indices) {
- withResource(makeTableForIndex(i)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- } finally {
- splitVectors.foreach(array => array.safeClose())
- }
- } else {
- withResource(GpuColumnVector.from(gpuCB)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- buffers.toList
- }
- }
-
- private def writeTableToCachedBatch(
- table: Table,
- schema: StructType): ParquetBufferConsumer = {
- val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
- val opts = GpuParquetFileFormat
- .parquetWriterOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
- .withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
- withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
- writer.write(table)
- }
- buffer
- }
-
- /**
- * This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying 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.
- */
- def gpuConvertCachedBatchToColumnarBatch(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- conf: SQLConf): RDD[ColumnarBatch] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- convertCachedBatchToColumnarInternal(
- input,
- cachedSchemaWithNames,
- selectedSchemaWithNames,
- newSelectedAttributes)
- }
-
- private def convertCachedBatchToColumnarInternal(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- originalSelectedAttributes: Seq[Attribute]): RDD[ColumnarBatch] = {
-
- val cbRdd: RDD[ColumnarBatch] = input.map {
- case parquetCB: ParquetCachedBatch =>
- val parquetOptions = ParquetOptions.builder()
- .includeColumn(selectedAttributes.map(_.name).asJavaCollection).build()
- withResource(Table.readParquet(parquetOptions, parquetCB.buffer, 0,
- parquetCB.sizeInBytes)) { table =>
- withResource {
- for (i <- 0 until table.getNumberOfColumns) yield {
- ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(table.getColumn(i),
- originalSelectedAttributes(i).dataType,
- (dataType, _) => dataType match {
- case d: DecimalType if d.scale < 0 => true
- case _ => false
- },
- (dataType, cv) => {
- //TODO: why do we have to copy to a vector
- dataType match {
- case d: DecimalType =>
- withResource(cv.bitCastTo(DecimalUtil.createCudfDecimal(d))) {
- _.copyToColumnVector()
- }
- case _ =>
- throw new IllegalStateException("We don't cast any type besides Decimal " +
- "with scale < 0")
- }
- }
- )
- }
- } { col =>
- withResource(new Table(col: _*)) { t =>
- GpuColumnVector.from(t, originalSelectedAttributes.map(_.dataType).toArray)
- }
- }
- }
- case _ =>
- throw new IllegalStateException("I don't know how to convert this batch")
- }
- cbRdd
- }
-
- private def getSelectedSchemaFromCachedSchema(
- selectedAttributes: Seq[Attribute],
- cacheAttributes: Seq[Attribute]): Seq[Attribute] = {
- selectedAttributes.map {
- a => cacheAttributes(cacheAttributes.map(_.exprId).indexOf(a.exprId))
- }
- }
-
- /**
- * 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] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val rapidsConf = new RapidsConf(conf)
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- if (rapidsConf.isSqlEnabled &&
- isSchemaSupportedByCudf(cachedSchemaWithNames)) {
- val batches = convertCachedBatchToColumnarInternal(input, cachedSchemaWithNames,
- selectedSchemaWithNames, newSelectedAttributes)
- val cbRdd = batches.map(batch => {
- withResource(batch) { gpuBatch =>
- val cols = GpuColumnVector.extractColumns(gpuBatch)
- new ColumnarBatch(cols.safeMap(_.copyToHost()).toArray, gpuBatch.numRows())
- }
- })
- cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter))
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, newSelectedAttributes, broadcastedConf).getColumnBatchIterator
- }
- }
- }
- }
-
- /**
- * 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 (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes)
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, selectedAttributes, broadcastedConf).getInternalRowIterator
- }
- }
- }
-
- private abstract class UnsupportedDataHandlerIterator extends Iterator[InternalRow] {
-
- def handleInternalRow(schema: Seq[Attribute], row: InternalRow, newRow: InternalRow): Unit
-
- def handleInterval(data: SpecializedGetters, index: Int): Any
-
- def handleStruct(
- data: InternalRow,
- origSchema: StructType,
- supportedSchema: StructType): InternalRow = {
- val structRow = InternalRow.fromSeq(supportedSchema)
- handleInternalRow(origSchema.map(field =>
- AttributeReference(field.name, field.dataType, field.nullable)()), data, structRow)
- structRow
- }
-
- def handleMap(
- keyType: DataType,
- valueType: DataType,
- mapData: MapData): MapData = {
- val keyData = mapData.keyArray()
- val newKeyData = handleArray(keyType, keyData)
- val valueData = mapData.valueArray()
- val newValueData = handleArray(valueType, valueData)
- new ArrayBasedMapData(newKeyData, newValueData)
- }
-
- def handleArray(
- dataType: DataType,
- arrayData: ArrayData): ArrayData = {
- dataType match {
- case s@StructType(_) =>
- val listBuffer = new ListBuffer[InternalRow]()
- val supportedSchema = mapping(dataType).asInstanceOf[StructType]
- arrayData.foreach(supportedSchema, (_, data) => {
- val structRow =
- handleStruct(data.asInstanceOf[InternalRow], s, s)
- listBuffer += structRow.copy()
- })
- new GenericArrayData(listBuffer)
-
- case ArrayType(elementType, _) =>
- val arrayList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val subArrayData = arrayData.getArray(i)
- arrayList.append(handleArray(elementType, subArrayData))
- }
- new GenericArrayData(arrayList)
-
- case m@MapType(_, _, _) =>
- val mapList =
- new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val mapData = arrayData.getMap(i)
- mapList.append(handleMap(m.keyType, m.valueType, mapData))
- }
- new GenericArrayData(mapList)
-
- case CalendarIntervalType =>
- val citList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val citRow = handleInterval(arrayData, i)
- citList += citRow
- }
- new GenericArrayData(citList)
-
- case _ =>
- arrayData
- }
- }
- }
-
- /**
- * Consumes the Iterator[CachedBatch] to return either Iterator[ColumnarBatch] or
- * Iterator[InternalRow]
- */
- private class CachedBatchIteratorConsumer(
- cbIter: Iterator[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- origCacheSchema: Seq[Attribute],
- origRequestedSchema: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val hadoopConf: Configuration = getHadoopConf(origRequestedSchema.toStructType, conf)
- val options: ParquetReadOptions = HadoopReadOptions.builder(hadoopConf).build()
- /**
- * We are getting this method using reflection because its a package-private
- */
- val readBatchMethod: Method =
- classOf[VectorizedColumnReader].getDeclaredMethod("readBatch", Integer.TYPE,
- classOf[WritableColumnVector])
- readBatchMethod.setAccessible(true)
-
- def getInternalRowIterator: Iterator[InternalRow] = {
-
- /**
- * This iterator converts an iterator[CachedBatch] to an iterator[InternalRow].
- *
- * This makes it unlike a regular iterator because CachedBatch => InternalRow* is a 1-n
- * relation. The way we have implemented this is to first go through the
- * iterator[CachedBatch] (cbIter) to look for a valid iterator (iter) i.e. hasNext() => true.
- * Then every time next() is called we return a single InternalRow from iter. When
- * iter.hasNext() => false, we find the next valid iterator in cbIter and the process
- * continues as above.
- */
- new Iterator[InternalRow]() {
-
- var iter: Iterator[InternalRow] = _
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = convertCachedBatchToInternalRowIter
- }
- iter != null && iter.hasNext
- }
-
- override def next(): InternalRow = {
- // will return the next InternalRow if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- /**
- * This method converts a CachedBatch to an iterator of InternalRows.
- */
- private def convertCachedBatchToInternalRowIter: Iterator[InternalRow] = {
- val parquetCachedBatch = cbIter.next().asInstanceOf[ParquetCachedBatch]
- val inputFile = new ByteArrayInputFile(parquetCachedBatch.buffer)
- withResource(ParquetFileReader.open(inputFile, options)) { parquetFileReader =>
- val parquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val hasUnsupportedType = origCacheSchema.exists { field =>
- !isTypeSupportedByParquet(field.dataType)
- }
-
- val unsafeRows = new ArrayBuffer[InternalRow]
- import org.apache.parquet.io.ColumnIOFactory
- var pages = parquetFileReader.readNextRowGroup()
- while (pages != null) {
- val rows = pages.getRowCount
- val columnIO = new ColumnIOFactory().getColumnIO(parquetSchema)
- val recordReader =
- columnIO.getRecordReader(pages, new ParquetRecordMaterializer(parquetSchema,
- cacheAttributes.toStructType,
- new ParquetToSparkSchemaConverter(hadoopConf), None /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED))
- for (_ <- 0 until rows.toInt) {
- val row = recordReader.read
- unsafeRows += row.copy()
- }
- pages = parquetFileReader.readNextRowGroup()
- }
-
- val iter = unsafeRows.iterator
- val unsafeProjection =
- GenerateUnsafeProjection.generate(selectedAttributes, cacheAttributes)
- if (hasUnsupportedType) {
- new UnsupportedDataHandlerIterator() {
- val wrappedIter: Iterator[InternalRow] = iter
- val newRow = new GenericInternalRow(cacheAttributes.length)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- //read a row and convert it to what the caller is expecting
- val row = wrappedIter.next()
- handleInternalRow(origCacheSchema, row, newRow)
- val unsafeProjection =
- GenerateUnsafeProjection.generate(origRequestedSchema, origCacheSchema)
- unsafeProjection.apply(newRow)
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): CalendarInterval = {
- if (data.isNullAt(index)) {
- null
- } else {
- val structData = data.getStruct(index, 3)
- new CalendarInterval(structData.getInt(0),
- structData.getInt(1), structData.getLong(2))
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val supportedSchema = mapping(dataType)
- .asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, supportedSchema.size), s, s)
- newRow.update(index, structRow)
-
- case a@ArrayType(_, _) =>
- val arrayData = row.getArray(index)
- newRow.update(index, handleArray(a.elementType, arrayData))
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- newRow.update(index, handleMap(keyType, valueType, mapData))
-
- case CalendarIntervalType =>
- val interval = handleInterval(row, index)
- if (interval == null) {
- newRow.setNullAt(index)
- } else {
- newRow.setInterval(index, interval)
- }
- case d: DecimalType =>
- if (row.isNullAt(index)) {
- newRow.setDecimal(index, null, d.precision)
- } else {
- val dec = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- Decimal(row.getInt(index).toLong, d.precision, d.scale)
- } else {
- Decimal(row.getLong(index), d.precision, d.scale)
- }
- newRow.update(index, dec)
- }
- case NullType =>
- newRow.setNullAt(index)
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- } else {
- iter.map(unsafeProjection)
- }
- }
- }
- }
- }
-
- private class CurrentBatchIterator(val parquetCachedBatch: ParquetCachedBatch)
- extends Iterator[ColumnarBatch] with AutoCloseable {
-
- val capacity = conf.parquetVectorizedReaderBatchSize
- var columnReaders: Array[VectorizedColumnReader] = _
- val columnVectors: Array[OffHeapColumnVector] =
- OffHeapColumnVector.allocateColumns(capacity, selectedAttributes.toStructType)
- val columnarBatch = new ColumnarBatch(columnVectors
- .asInstanceOf[Array[vectorized.ColumnVector]])
- var rowsReturned: Long = 0L
- var numBatched = 0
- var batchIdx = 0
- var totalCountLoadedSoFar: Long = 0
- val parquetFileReader =
- ParquetFileReader.open(new ByteArrayInputFile(parquetCachedBatch.buffer), options)
- val (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache) = {
- val parquetToSparkSchemaConverter = new ParquetToSparkSchemaConverter(hadoopConf)
- // we are getting parquet schema and then converting it to catalyst schema
- // because catalyst schema that we get from Spark doesn't have the exact schema expected
- // by the columnar parquet reader
- val inMemCacheParquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val inMemCacheSparkSchema = parquetToSparkSchemaConverter.convert(inMemCacheParquetSchema)
-
- val totalRowCount = parquetFileReader.getRowGroups.asScala.map(_.getRowCount).sum
- val sparkToParquetSchemaConverter = new SparkToParquetSchemaConverter(hadoopConf)
- val inMemReqSparkSchema = StructType(selectedAttributes.toStructType.map { field =>
- inMemCacheSparkSchema.fields(inMemCacheSparkSchema.fieldIndex(field.name))
- })
- val inMemReqParquetSchema = sparkToParquetSchemaConverter.convert(inMemReqSparkSchema)
- val columnsRequested: util.List[ColumnDescriptor] = inMemReqParquetSchema.getColumns
-
- val reqSparkSchemaInCacheOrder = StructType(inMemCacheSparkSchema.filter(f =>
- inMemReqSparkSchema.fields.exists(f0 => f0.name.equals(f.name))))
-
- // There could be a case especially in a distributed environment where the requestedSchema
- // and cacheSchema are not in the same order. We need to create a map so we can guarantee
- // that we writing to the correct columnVector
- val cacheSchemaToReqSchemaMap: Map[Int, Int] =
- reqSparkSchemaInCacheOrder.indices.map { index =>
- index -> inMemReqSparkSchema.fields.indexOf(reqSparkSchemaInCacheOrder.fields(index))
- }.toMap
-
- val reqParquetSchemaInCacheOrder =
- sparkToParquetSchemaConverter.convert(reqSparkSchemaInCacheOrder)
-
- // reset spark schema calculated from parquet schema
- hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, inMemReqSparkSchema.json)
- hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, inMemReqSparkSchema.json)
-
- val columnsInCache: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
- val typesInCache: util.List[Type] = reqParquetSchemaInCacheOrder.asGroupType.getFields
- val missingColumns = new Array[Boolean](inMemReqParquetSchema.getFieldCount)
-
- // initialize missingColumns to cover the case where requested column isn't present in the
- // cache, which should never happen but just in case it does
- val paths: util.List[Array[String]] = inMemReqParquetSchema.getPaths
-
- for (i <- 0 until inMemReqParquetSchema.getFieldCount) {
- val t = inMemReqParquetSchema.getFields.get(i)
- if (!t.isPrimitive || t.isRepetition(Type.Repetition.REPEATED)) {
- throw new UnsupportedOperationException("Complex types not supported.")
- }
- val colPath = paths.get(i)
- if (inMemCacheParquetSchema.containsPath(colPath)) {
- val fd = inMemCacheParquetSchema.getColumnDescription(colPath)
- if (!(fd == columnsRequested.get(i))) {
- throw new UnsupportedOperationException("Schema evolution not supported.")
- }
- missingColumns(i) = false
- } else {
- if (columnsRequested.get(i).getMaxDefinitionLevel == 0) {
- // Column is missing in data but the required data is non-nullable.
- // This file is invalid.
- throw new IOException(s"Required column is missing in data file: ${colPath.toList}")
- }
- missingColumns(i) = true
- }
- }
-
- for (i <- missingColumns.indices) {
- if (missingColumns(i)) {
- columnVectors(i).putNulls(0, capacity)
- columnVectors(i).setIsConstant()
- }
- }
-
- (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache)
- }
-
- @throws[IOException]
- def checkEndOfRowGroup(): Unit = {
- if (rowsReturned != totalCountLoadedSoFar) return
- val pages = parquetFileReader.readNextRowGroup
- if (pages == null) {
- throw new IOException("expecting more rows but reached last" +
- " block. Read " + rowsReturned + " out of " + totalRowCount)
- }
- columnReaders = new Array[VectorizedColumnReader](columnsRequested.size)
- for (i <- 0 until columnsRequested.size) {
- if (!missingColumns(i)) {
- columnReaders(i) =
- // TODO - https://github.com/NVIDIA/spark-rapids/issues/3265
- // I added extra boolean parameter as false but I have not
- // tested and we need to follow up
- new VectorizedColumnReader(
- columnsInCache.get(i),
- typesInCache.get(i).getOriginalType,
- pages.getPageReader(columnsInCache.get(i)),
- null /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED.toString,
- LegacyBehaviorPolicy.EXCEPTION.toString,
- false)
- }
- }
- totalCountLoadedSoFar += pages.getRowCount
- }
-
- /**
- * Read the next RowGroup and read each column and return the columnarBatch
- */
- def nextBatch: Boolean = {
- for (vector <- columnVectors) {
- vector.reset()
- }
- columnarBatch.setNumRows(0)
- if (rowsReturned >= totalRowCount) return false
- checkEndOfRowGroup()
- val num = Math.min(capacity.toLong, totalCountLoadedSoFar - rowsReturned).toInt
- for (i <- columnReaders.indices) {
- if (columnReaders(i) != null) {
- readBatchMethod.invoke(columnReaders(i), num.asInstanceOf[AnyRef],
- columnVectors(cacheSchemaToReqSchemaMap(i)).asInstanceOf[AnyRef])
- }
- }
- rowsReturned += num
- columnarBatch.setNumRows(num)
- numBatched = num
- batchIdx = 0
- true
- }
-
- override def hasNext: Boolean = rowsReturned < totalRowCount
-
- override def next(): ColumnarBatch = {
- if (nextBatch) {
- // FYI, A very IMPORTANT thing to note is that we are returning the columnar batch
- // as-is i.e. this batch has NullTypes saved as IntegerTypes with null values. The
- // way Spark optimizes the read of NullTypes makes this work without having to rip out
- // the IntegerType column to be replaced by a NullType column. This could change in
- // future and will affect this code.
- columnarBatch
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => {
- close()
- })
-
- override def close(): Unit = {
- parquetFileReader.close()
- }
- }
-
- /**
- * This method returns a ColumnarBatch iterator over a CachedBatch.
- * Each CachedBatch => ColumnarBatch is a 1-1 conversion so its pretty straight forward
- */
- def getColumnBatchIterator: Iterator[ColumnarBatch] = new Iterator[ColumnarBatch] {
- var iter = getIterator
-
- def getIterator: Iterator[ColumnarBatch] = {
- if (!cbIter.hasNext) {
- Iterator.empty
- } else {
- new CurrentBatchIterator(cbIter.next().asInstanceOf[ParquetCachedBatch])
- }
- }
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = getIterator
- }
- iter != null && iter.hasNext
- }
-
- override def next(): ColumnarBatch = {
- // will return the next ColumnarBatch if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
- }
- }
-
- private def getConfFromMap(sharedConf: Broadcast[Map[String, String]]): SQLConf = {
- val conf = new SQLConf()
- sharedConf.value.foreach { case (k, v) => conf.setConfString(k, v) }
- conf
- }
-
- private val intervalStructType = new StructType()
- .add("_days", IntegerType)
- .add("_months", IntegerType)
- .add("_ms", LongType)
-
- def getBytesAllowedPerBatch(conf: SQLConf): Long = {
- val gpuBatchSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf)
- // we are rough estimating 0.5% as meta_data_size. we can do better estimation in future
- val approxMetaDataSizeBytes = gpuBatchSize * 0.5/100
- (gpuBatchSize - approxMetaDataSizeBytes).toLong
- }
-
- /**
- * This is a private helper class to return Iterator to convert InternalRow or ColumnarBatch to
- * CachedBatch. There is no type checking so if the type of T is anything besides InternalRow
- * or ColumnarBatch then the behavior is undefined.
- *
- * @param iter - an iterator over InternalRow or ColumnarBatch
- * @param cachedAttributes - Schema of the cached batch
- * @param sharedConf - SQL conf
- * @tparam T - Strictly either InternalRow or ColumnarBatch
- */
- private[rapids] class CachedBatchIteratorProducer[T](
- iter: Iterator[T],
- cachedAttributes: Seq[Attribute],
- origCachedAttributes: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val hadoopConf: Configuration = getHadoopConf(cachedAttributes.toStructType, conf)
-
- def getInternalRowToCachedBatchIterator: Iterator[CachedBatch] = {
- new InternalRowToCachedBatchIterator
- }
-
- def getColumnarBatchToCachedBatchIterator: Iterator[CachedBatch] = {
- new ColumnarBatchToCachedBatchIterator
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[InternalRow]. This is a n-1
- * relationship. Each partition represents a single parquet file, so we encode it
- * and return the CachedBatch when next is called.
- */
- class InternalRowToCachedBatchIterator extends Iterator[CachedBatch]() {
-
- var parquetOutputFileFormat = new ParquetOutputFileFormat()
-
- // For testing only
- private[rapids] def setParquetOutputFileFormat(p: ParquetOutputFileFormat): Unit = {
- parquetOutputFileFormat = p
- }
-
- // is there a type that spark doesn't support by default in the schema?
- val hasUnsupportedType: Boolean = origCachedAttributes.exists { attribute =>
- !isTypeSupportedByParquet(attribute.dataType)
- }
-
- def getIterator: Iterator[InternalRow] = {
- if (!hasUnsupportedType) {
- iter.asInstanceOf[Iterator[InternalRow]]
- } else {
- new UnsupportedDataHandlerIterator {
-
- val wrappedIter: Iterator[InternalRow] = iter.asInstanceOf[Iterator[InternalRow]]
-
- val newRow: InternalRow = InternalRow.fromSeq(cachedAttributes)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- val row = wrappedIter.next()
- handleInternalRow(origCachedAttributes, row, newRow)
- newRow
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): InternalRow = {
- val citRow = InternalRow(IntegerType, IntegerType, LongType)
- if (data.isNullAt(index)) {
- null
- } else {
- val cit = data.getInterval(index)
- citRow.setInt(0, cit.months)
- citRow.setInt(1, cit.days)
- citRow.setLong(2, cit.microseconds)
- citRow
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val newSchema = mapping(dataType).asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, s.fields.length), s, newSchema)
- newRow.update(index, structRow)
-
- case ArrayType(arrayDataType, _) =>
- val arrayData = row.getArray(index)
- val newArrayData = handleArray(arrayDataType, arrayData)
- newRow.update(index, newArrayData)
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- val map = handleMap(keyType, valueType, mapData)
- newRow.update(index, map)
-
- case CalendarIntervalType =>
- val structData: InternalRow = handleInterval(row, index)
- if (structData == null) {
- newRow.setNullAt(index)
- } else {
- newRow.update(index, structData)
- }
-
- case d: DecimalType if d.scale < 0 =>
- if (d.precision <= Decimal.MAX_INT_DIGITS) {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong.toInt)
- } else {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong)
- }
-
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- }
- }
-
- override def hasNext: Boolean = queue.nonEmpty || iter.hasNext
-
- private val queue = new mutable.Queue[CachedBatch]()
-
- //estimate the size of a row
- val estimatedSize: Int = cachedAttributes.map { attr =>
- attr.dataType.defaultSize
- }.sum
-
- override def next(): CachedBatch = {
- if (queue.isEmpty) {
- // to store a row if we have read it but there is no room in the parquet file to put it
- // we will put it in the next CachedBatch
- var leftOverRow: Option[InternalRow] = None
- val rowIterator = getIterator
- while (rowIterator.hasNext || leftOverRow.nonEmpty) {
- // Each partition will be a single parquet file
- var rows = 0
- // at least a single block
- val stream = new ByteArrayOutputStream(ByteArrayOutputFile.BLOCK_SIZE)
- val outputFile: OutputFile = new ByteArrayOutputFile(stream)
- conf.setConfString(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
- val recordWriter = SQLConf.withExistingConf(conf) {
- parquetOutputFileFormat.getRecordWriter(outputFile, hadoopConf)
- }
- var totalSize = 0
- while ((rowIterator.hasNext || leftOverRow.nonEmpty)
- && totalSize < bytesAllowedPerBatch) {
-
- val row = if (leftOverRow.nonEmpty) {
- val a = leftOverRow.get
- leftOverRow = None // reset value
- a
- } else {
- rowIterator.next()
- }
- totalSize += {
- row match {
- case r: UnsafeRow =>
- r.getSizeInBytes
- case _ =>
- estimatedSize
- }
- }
- if (totalSize <= bytesAllowedPerBatch) {
- rows += 1
- if (rows < 0) {
- throw new IllegalStateException("CachedBatch doesn't support rows larger " +
- "than Int.MaxValue")
- }
- recordWriter.write(null, row)
- } else {
- leftOverRow = Some(if (row.isInstanceOf[UnsafeRow]) {
- row.copy()
- } else {
- row
- })
- }
- }
- // passing null as context isn't used in this method
- recordWriter.close(null)
- queue += ParquetCachedBatch(rows, stream.toByteArray)
- }
- }
- queue.dequeue()
- }
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[ColumnarBatch]. This is a 1-1
- * relationship. Each ColumnarBatch is converted to a single ParquetCachedBatch when next()
- * is called on this iterator
- */
- class ColumnarBatchToCachedBatchIterator extends InternalRowToCachedBatchIterator {
- override def getIterator: Iterator[InternalRow] = {
-
- new Iterator[InternalRow] {
- // We have to check for null context because of the unit test
- Option(TaskContext.get).foreach(_.addTaskCompletionListener[Unit](_ => hostBatch.close()))
-
- val batch: ColumnarBatch = iter.asInstanceOf[Iterator[ColumnarBatch]].next
- val hostBatch = if (batch.column(0).isInstanceOf[GpuColumnVector]) {
- withResource(batch) { batch =>
- new ColumnarBatch(batch.safeMap(_.copyToHost()).toArray, batch.numRows())
- }
- } else {
- batch
- }
-
- val rowIterator = hostBatch.rowIterator().asScala
-
- override def next: InternalRow = rowIterator.next
-
- override def hasNext: Boolean = rowIterator.hasNext
-
- }
- }
- }
-
- }
-
- val mapping = new mutable.HashMap[DataType, DataType]()
-
- def getSupportedDataType(curId: AtomicLong, dataType: DataType): DataType = {
- dataType match {
- case CalendarIntervalType =>
- intervalStructType
- case NullType =>
- ByteType
- case s: StructType =>
- val newStructType = StructType(
- s.indices.map { index =>
- StructField(curId.getAndIncrement().toString,
- getSupportedDataType(curId, s.fields(index).dataType), s.fields(index).nullable,
- s.fields(index).metadata)
- })
- mapping.put(s, newStructType)
- newStructType
- case a@ArrayType(elementType, nullable) =>
- val newArrayType =
- ArrayType(getSupportedDataType(curId, elementType), nullable)
- mapping.put(a, newArrayType)
- newArrayType
- case m@MapType(keyType, valueType, nullable) =>
- val newKeyType = getSupportedDataType(curId, keyType)
- val newValueType = getSupportedDataType(curId, valueType)
- val mapType = MapType(newKeyType, newValueType, nullable)
- mapping.put(m, mapType)
- mapType
- case d: DecimalType if d.scale < 0 =>
- val newType = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- IntegerType
- } else {
- LongType
- }
- newType
- case _ =>
- dataType
- }
- }
-
- private def getSupportedSchemaFromUnsupported(
- cachedAttributes: Seq[Attribute],
- requestedAttributes: Seq[Attribute] = Seq.empty): (Seq[Attribute], Seq[Attribute]) = {
-
- // We only handle CalendarIntervalType, Decimals and NullType ATM convert it to a supported type
- val curId = new AtomicLong()
- val newCachedAttributes = cachedAttributes.map {
- attribute => val name = s"_col${curId.getAndIncrement()}"
- attribute.dataType match {
- case CalendarIntervalType =>
- AttributeReference(name, intervalStructType,
- attribute.nullable, metadata = attribute.metadata)(attribute.exprId)
- .asInstanceOf[Attribute]
- case NullType =>
- AttributeReference(name, DataTypes.ByteType,
- nullable = true, metadata =
- attribute.metadata)(attribute.exprId).asInstanceOf[Attribute]
- case StructType(_) | ArrayType(_, _) | MapType(_, _, _) | DecimalType() =>
- AttributeReference(name,
- getSupportedDataType(curId, attribute.dataType),
- attribute.nullable, attribute.metadata)(attribute.exprId)
- case _ =>
- attribute.withName(name)
- }
- }
-
- val newRequestedAttributes =
- getSelectedSchemaFromCachedSchema(requestedAttributes, newCachedAttributes)
-
- (newCachedAttributes, newRequestedAttributes)
- }
-
- private def getHadoopConf(requestedSchema: StructType,
- sqlConf: SQLConf): Configuration = {
-
- val hadoopConf = new Configuration(false)
- hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
- hadoopConf.set(
- ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- ParquetWriteSupport.SPARK_ROW_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sqlConf.sessionLocalTimeZone)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, false)
- hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, false)
-
- hadoopConf.set(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
-
- hadoopConf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key,
- SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, false)
-
- hadoopConf.set(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString)
-
- ParquetWriteSupport.setSchema(requestedSchema, hadoopConf)
-
- hadoopConf
- }
-
- /**
- * 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- val structSchema = schemaWithUnambiguousNames.toStructType
- val converters = new GpuRowToColumnConverter(structSchema)
- val columnarBatchRdd = input.mapPartitions(iter => {
- new RowToColumnarIterator(iter, structSchema, RequireSingleBatch, converters)
- })
- columnarBatchRdd.flatMap(cb => {
- withResource(cb)(cb => compressColumnarBatchWithParquet(cb, structSchema,
- schema.toStructType, bytesAllowedPerBatch))
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[InternalRow](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getInternalRowToCachedBatchIterator
- }
- }
- }
-
- override def buildFilter(
- predicates: Seq[Expression],
- cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
- //essentially a noop
- (_: Int, b: Iterator[CachedBatch]) => b
- }
-}
-
-/**
- * Similar to ParquetFileFormat
- */
-private[rapids] class ParquetOutputFileFormat {
-
- def getRecordWriter(output: OutputFile, conf: Configuration): RecordWriter[Void, InternalRow] = {
- import ParquetOutputFormat._
-
- val blockSize = getLongBlockSize(conf)
- val maxPaddingSize =
- conf.getInt(MAX_PADDING_BYTES, ParquetWriter.MAX_PADDING_SIZE_DEFAULT)
- val validating = getValidation(conf)
-
- val writeSupport = new ParquetWriteSupport().asInstanceOf[WriteSupport[InternalRow]]
- val init = writeSupport.init(conf)
- val writer = new ParquetFileWriter(output, init.getSchema,
- Mode.CREATE, blockSize, maxPaddingSize)
- writer.start()
-
- val writerVersion =
- ParquetProperties.WriterVersion.fromString(conf.get(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString))
-
- val codecFactory = new CodecFactory(conf, getPageSize(conf))
-
- new ParquetRecordWriter[InternalRow](writer, writeSupport, init.getSchema,
- init.getExtraMetaData, blockSize, getPageSize(conf),
- codecFactory.getCompressor(CompressionCodecName.UNCOMPRESSED), getDictionaryPageSize(conf),
- getEnableDictionary(conf), validating, writerVersion,
- ParquetOutputFileFormat.getMemoryManager(conf))
-
- }
-}
-
-private object ParquetOutputFileFormat {
- var memoryManager: MemoryManager = _
- val DEFAULT_MEMORY_POOL_RATIO: Float = 0.95f
- val DEFAULT_MIN_MEMORY_ALLOCATION: Long = 1 * 1024 * 1024 // 1MB
-
- def getMemoryManager(conf: Configuration): MemoryManager = {
- synchronized {
- if (memoryManager == null) {
- import ParquetOutputFormat._
- val maxLoad = conf.getFloat(MEMORY_POOL_RATIO, DEFAULT_MEMORY_POOL_RATIO)
- val minAllocation = conf.getLong(MIN_MEMORY_ALLOCATION, DEFAULT_MIN_MEMORY_ALLOCATION)
- memoryManager = new MemoryManager(maxLoad, minAllocation)
- }
- }
- memoryManager
- }
-}
diff --git a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala
index 876d568d622..407ca85cbf2 100644
--- a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala
+++ b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/Spark311CDHShims.scala
@@ -18,6 +18,7 @@ package com.nvidia.spark.rapids.shims.spark311cdh
import java.net.URI
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.spark311cdh.RapidsShuffleManager
@@ -26,6 +27,7 @@ import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.rapids.shims.spark311cdh._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
class Spark311CDHShims extends SparkBaseShims {
diff --git a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/SparkBaseShims.scala b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/SparkBaseShims.scala
index 8d5c16a81da..cde11f413b4 100644
--- a/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/SparkBaseShims.scala
+++ b/shims/spark311cdh/src/main/scala/com/nvidia/spark/rapids/shims/spark311cdh/SparkBaseShims.scala
@@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.shims.spark311cdh
import java.net.URI
import java.nio.ByteBuffer
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.v2.Spark30XShims
import org.apache.arrow.memory.ReferenceManager
@@ -58,6 +59,7 @@ import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuB
import org.apache.spark.sql.rapids.execution.python.GpuPythonUDF
import org.apache.spark.sql.rapids.execution.python.shims.spark311cdh._
import org.apache.spark.sql.rapids.shims.spark311cdh._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
diff --git a/shims/spark311cdh/src/main/scala/org/apache/spark/sql/rapids/shims/spark311cdh/GpuInMemoryTableScanExec.scala b/shims/spark311cdh/src/main/scala/org/apache/spark/sql/rapids/shims/spark311cdh/GpuInMemoryTableScanExec.scala
deleted file mode 100644
index 06365ff62a2..00000000000
--- a/shims/spark311cdh/src/main/scala/org/apache/spark/sql/rapids/shims/spark311cdh/GpuInMemoryTableScanExec.scala
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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 org.apache.spark.sql.rapids.shims.spark311cdh
-
-import com.nvidia.spark.rapids.{GpuExec, GpuMetric}
-import com.nvidia.spark.rapids.shims.spark311cdh.ParquetCachedBatchSerializer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.physical.Partitioning
-import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
-import org.apache.spark.sql.vectorized.ColumnarBatch
-
-case class GpuInMemoryTableScanExec(
- attributes: Seq[Attribute],
- predicates: Seq[Expression],
- @transient relation: InMemoryRelation) extends LeafExecNode with GpuExec {
-
- override val nodeName: String = {
- relation.cacheBuilder.tableName match {
- case Some(_) =>
- "Scan " + relation.cacheBuilder.cachedName
- case _ =>
- super.nodeName
- }
- }
-
- override def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren
-
- override def doCanonicalize(): SparkPlan =
- copy(attributes = attributes.map(QueryPlan.normalizeExpressions(_, relation.output)),
- predicates = predicates.map(QueryPlan.normalizeExpressions(_, relation.output)),
- relation = relation.canonicalized.asInstanceOf[InMemoryRelation])
-
- override def vectorTypes: Option[Seq[String]] =
- relation.cacheBuilder.serializer.vectorTypes(attributes, conf)
-
- private lazy val columnarInputRDD: RDD[ColumnarBatch] = {
- val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS)
- val buffers = filteredCachedBatches()
- relation.cacheBuilder.serializer.asInstanceOf[ParquetCachedBatchSerializer]
- .gpuConvertCachedBatchToColumnarBatch(
- buffers,
- relation.output,
- attributes,
- conf).map { cb =>
- numOutputRows += cb.numRows()
- cb
- }
- }
-
- override def output: Seq[Attribute] = attributes
-
- private def updateAttribute(expr: Expression): Expression = {
- // attributes can be pruned so using relation's output.
- // E.g., relation.output is [id, item] but this scan's output can be [item] only.
- val attrMap = AttributeMap(relation.cachedPlan.output.zip(relation.output))
- expr.transform {
- case attr: Attribute => attrMap.getOrElse(attr, attr)
- }
- }
-
- // The cached version does not change the outputPartitioning of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputPartitioning: Partitioning = {
- relation.cachedPlan.outputPartitioning match {
- case e: Expression => updateAttribute(e).asInstanceOf[Partitioning]
- case other => other
- }
- }
-
- // The cached version does not change the outputOrdering of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputOrdering: Seq[SortOrder] =
- relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder])
-
- lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled
-
- // Accumulators used for testing purposes
- lazy val readPartitions = sparkContext.longAccumulator
- lazy val readBatches = sparkContext.longAccumulator
-
- private def filteredCachedBatches() = {
- // Right now just return the batch without filtering
- relation.cacheBuilder.cachedColumnBuffers
- }
-
- protected override def doExecute(): RDD[InternalRow] = {
- throw new UnsupportedOperationException("This Exec only deals with Columnar Data")
- }
-
- protected override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- columnarInputRDD
- }
-}
diff --git a/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/ParquetCachedBatchSerializer.scala b/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/ParquetCachedBatchSerializer.scala
deleted file mode 100644
index 80419082b20..00000000000
--- a/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/ParquetCachedBatchSerializer.scala
+++ /dev/null
@@ -1,1521 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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.spark311db
-
-import java.io.{InputStream, IOException}
-import java.lang.reflect.Method
-import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-
-import ai.rapids.cudf._
-import ai.rapids.cudf.ParquetWriterOptions.StatisticsFrequency
-import com.nvidia.spark.rapids._
-import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
-import com.nvidia.spark.rapids.RapidsPluginImplicits._
-import java.util
-import org.apache.commons.io.output.ByteArrayOutputStream
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.RecordWriter
-import org.apache.parquet.{HadoopReadOptions, ParquetReadOptions}
-import org.apache.parquet.column.{ColumnDescriptor, ParquetProperties}
-import org.apache.parquet.hadoop.{CodecFactory, MemoryManager, ParquetFileReader, ParquetFileWriter, ParquetInputFormat, ParquetOutputFormat, ParquetRecordWriter, ParquetWriter}
-import org.apache.parquet.hadoop.ParquetFileWriter.Mode
-import org.apache.parquet.hadoop.api.WriteSupport
-import org.apache.parquet.hadoop.metadata.CompressionCodecName
-import org.apache.parquet.io.{DelegatingPositionOutputStream, DelegatingSeekableInputStream, InputFile, OutputFile, PositionOutputStream, SeekableInputStream}
-import org.apache.parquet.schema.{MessageType, Type}
-
-import org.apache.spark.TaskContext
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{vectorized, SparkSession}
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericInternalRow, SpecializedGetters, UnsafeRow}
-import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
-import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
-import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer}
-import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetToSparkSchemaConverter, ParquetWriteSupport, SparkToParquetSchemaConverter, VectorizedColumnReader}
-import org.apache.spark.sql.execution.datasources.parquet.rapids.shims.spark311db.ParquetRecordMaterializer
-import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.unsafe.types.CalendarInterval
-
-/**
- * copied from Spark org.apache.spark.util.ByteBufferInputStream
- */
-private class ByteBufferInputStream(private var buffer: ByteBuffer)
- extends InputStream {
-
- override def read(): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- buffer.get() & 0xFF
- }
- }
-
- override def read(dest: Array[Byte]): Int = {
- read(dest, 0, dest.length)
- }
-
- override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- val amountToGet = math.min(buffer.remaining(), length)
- buffer.get(dest, offset, amountToGet)
- amountToGet
- }
- }
-
- override def skip(bytes: Long): Long = {
- if (buffer != null) {
- val amountToSkip = math.min(bytes, buffer.remaining).toInt
- buffer.position(buffer.position() + amountToSkip)
- if (buffer.remaining() == 0) {
- cleanUp()
- }
- amountToSkip
- } else {
- 0L
- }
- }
-
- /**
- * Clean up the buffer, and potentially dispose of it using StorageUtils.dispose().
- */
- private def cleanUp(): Unit = {
- if (buffer != null) {
- buffer = null
- }
- }
-}
-
-class ByteArrayInputFile(buff: Array[Byte]) extends InputFile {
-
- override def getLength: Long = buff.length
-
- override def newStream(): SeekableInputStream = {
- val byteBuffer = ByteBuffer.wrap(buff)
- new DelegatingSeekableInputStream(new ByteBufferInputStream(byteBuffer)) {
- override def getPos: Long = byteBuffer.position()
-
- override def seek(newPos: Long): Unit = {
- if (newPos > Int.MaxValue || newPos < Int.MinValue) {
- throw new IllegalStateException("seek value is out of supported range " + newPos)
- }
- byteBuffer.position(newPos.toInt)
- }
- }
- }
-}
-
-private object ByteArrayOutputFile {
- val BLOCK_SIZE: Int = 32 * 1024 * 1024 // 32M
-}
-
-private class ByteArrayOutputFile(stream: ByteArrayOutputStream) extends OutputFile {
- override def create(blockSizeHint: Long): PositionOutputStream = {
- new DelegatingPositionOutputStream(stream) {
- var pos = 0
-
- override def getPos: Long = pos
-
- override def write(b: Int): Unit = {
- super.write(b)
- pos += Integer.BYTES
- }
-
- override def write(b: Array[Byte]): Unit = {
- super.write(b)
- pos += b.length
- }
-
- override def write(b: Array[Byte], off: Int, len: Int): Unit = {
- super.write(b, off, len)
- pos += len
- }
- }
- }
-
- override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream =
- throw new UnsupportedOperationException("Don't need to overwrite")
-
- override def supportsBlockSize(): Boolean = true
-
- override def defaultBlockSize(): Long = ByteArrayOutputFile.BLOCK_SIZE
-}
-
-private class ParquetBufferConsumer(val numRows: Int) extends HostBufferConsumer with
- AutoCloseable {
- @transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
- private var buffer: Array[Byte] = _
-
- override def handleBuffer(buffer: HostMemoryBuffer, len: Long): Unit = {
- offHeapBuffers += Tuple2(buffer, len)
- }
-
- def getBuffer: Array[Byte] = {
- if (buffer == null) {
- writeBuffers()
- }
- buffer
- }
-
- def close(): Unit = {
- if (buffer == null) {
- writeBuffers()
- }
- }
-
- private def writeBuffers(): Unit = {
- val toProcess = offHeapBuffers.dequeueAll(_ => true)
- // We are making sure the input is smaller than 2gb so the parquet written should never be more
- // than Int.MAX_SIZE.
- val bytes = toProcess.map(_._2).sum
-
- // for now assert bytes are less than Int.MaxValue
- assert(bytes <= Int.MaxValue)
- buffer = new Array(bytes.toInt)
- try {
- var offset: Int = 0
- toProcess.foreach(ops => {
- val origBuffer = ops._1
- val len = ops._2.toInt
- origBuffer.asByteBuffer().get(buffer, offset, len)
- offset = offset + len
- })
- } finally {
- toProcess.map(_._1).safeClose()
- }
- }
-}
-
-private object ParquetCachedBatch {
- def apply(parquetBuff: ParquetBufferConsumer): ParquetCachedBatch = {
- new ParquetCachedBatch(parquetBuff.numRows, parquetBuff.getBuffer)
- }
-}
-
-case class ParquetCachedBatch(
- numRows: Int,
- buffer: Array[Byte]) extends CachedBatch {
- override def sizeInBytes: Long = buffer.length
-}
-
-/**
- * Spark wants the producer to close the batch. We have a listener in this iterator that will close
- * the batch after the task is completed
- */
-private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
- Iterator[ColumnarBatch] {
- var cb: ColumnarBatch = _
-
- private def closeCurrentBatch(): Unit = {
- if (cb != null) {
- cb.close()
- cb = null
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: 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 = schema.fields.forall { f =>
- // only check spark b/c if we are on the GPU then we will be calling the gpu method regardless
- isTypeSupportedByColumnarSparkParquetWriter(f.dataType) || f.dataType == DataTypes.NullType
- }
-
- private def isTypeSupportedByColumnarSparkParquetWriter(dataType: DataType): Boolean = {
- // Columnar writer in Spark only supports AtomicTypes ATM
- dataType match {
- case TimestampType | StringType | BooleanType | DateType | BinaryType |
- DoubleType | FloatType | ByteType | IntegerType | LongType | ShortType => true
- case _: DecimalType => true
- case _ => false
- }
- }
-
- def isSchemaSupportedByCudf(schema: Seq[Attribute]): Boolean = {
- schema.forall(field => isSupportedByCudf(field.dataType))
- }
-
- def isSupportedByCudf(dataType: DataType): Boolean = {
- dataType match {
- // TODO: when arrays are supported for cudf writes add it here.
- // https://github.com/NVIDIA/spark-rapids/issues/2054
- case s: StructType => s.forall(field => isSupportedByCudf(field.dataType))
- case _ => GpuColumnVector.isNonNestedSupportedType(dataType)
- }
- }
-
- /**
- * This method checks if the datatype passed is officially supported by parquet.
- *
- * Please refer to https://github.com/apache/parquet-format/blob/master/LogicalTypes.md to see
- * the what types are supported by parquet
- */
- def isTypeSupportedByParquet(dataType: DataType): Boolean = {
- dataType match {
- case CalendarIntervalType | NullType => false
- case s: StructType => s.forall(field => isTypeSupportedByParquet(field.dataType))
- case ArrayType(elementType, _) => isTypeSupportedByParquet(elementType)
- case MapType(keyType, valueType, _) => isTypeSupportedByParquet(keyType) &&
- isTypeSupportedByParquet(valueType)
- case d: DecimalType if d.scale < 0 => false
- case _ => 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- val structSchema = schemaWithUnambiguousNames.toStructType
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- def putOnGpuIfNeeded(batch: ColumnarBatch): ColumnarBatch = {
- if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
- val s: StructType = structSchema
- val gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows()).build(batch.numRows())
- batch.close()
- gpuCB
- } else {
- batch
- }
- }
-
- input.flatMap(batch => {
- if (batch.numCols() == 0) {
- List(ParquetCachedBatch(batch.numRows(), new Array[Byte](0)))
- } else {
- withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
- compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
- bytesAllowedPerBatch)
- }
- }
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[ColumnarBatch](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getColumnarBatchToCachedBatchIterator
- }
- }
- }
-
- private[rapids] def compressColumnarBatchWithParquet(
- oldGpuCB: ColumnarBatch,
- schema: StructType,
- origSchema: StructType,
- bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
- val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
- oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
- .getBase.getDeviceMemorySize / oldGpuCB.numRows()
- }.sum
-
- val columns = for (i <- 0 until oldGpuCB.numCols()) yield {
- val gpuVector = oldGpuCB.column(i).asInstanceOf[GpuColumnVector]
- var dataType = origSchema(i).dataType
- val v = ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(gpuVector.getBase,
- origSchema(i).dataType,
- // we are checking for scale > 0 because cudf and spark refer to scales as opposites
- // e.g. scale = -3 in Spark is scale = 3 in cudf
- (_, cv) => cv.getType.isDecimalType && cv.getType.getScale > 0,
- (_, cv) => {
- if (cv.getType.isBackedByLong) {
- dataType = LongType
- cv.bitCastTo(DType.INT64)
- } else {
- dataType = IntegerType
- cv.bitCastTo(DType.INT32)
- }
- }
- )
- GpuColumnVector.from(v, schema(i).dataType)
- }
- withResource(new ColumnarBatch(columns.toArray, oldGpuCB.numRows())) { gpuCB =>
- val rowsAllowedInBatch = (bytesAllowedPerBatch / estimatedRowSize).toInt
- val splitIndices = scala.Range(rowsAllowedInBatch, gpuCB.numRows(), rowsAllowedInBatch)
- val buffers = new ListBuffer[ParquetCachedBatch]
- if (splitIndices.nonEmpty) {
- val splitVectors = new ListBuffer[Array[ColumnVector]]
- try {
- for (index <- 0 until gpuCB.numCols()) {
- splitVectors +=
- gpuCB.column(index).asInstanceOf[GpuColumnVector].getBase.split(splitIndices: _*)
- }
-
- // Splitting the table
- // e.g. T0 = {col1, col2,...,coln} => split columns into 'm' cols =>
- // T00= {splitCol1(0), splitCol2(0),...,splitColn(0)}
- // T01= {splitCol1(1), splitCol2(1),...,splitColn(1)}
- // ...
- // T0m= {splitCol1(m), splitCol2(m),...,splitColn(m)}
- def makeTableForIndex(i: Int): Table = {
- val columns = splitVectors.indices.map(j => splitVectors(j)(i))
- new Table(columns: _*)
- }
-
- for (i <- splitVectors.head.indices) {
- withResource(makeTableForIndex(i)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- } finally {
- splitVectors.foreach(array => array.safeClose())
- }
- } else {
- withResource(GpuColumnVector.from(gpuCB)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- buffers.toList
- }
- }
-
- private def writeTableToCachedBatch(
- table: Table,
- schema: StructType): ParquetBufferConsumer = {
- val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
- val opts = GpuParquetFileFormat
- .parquetWriterOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
- .withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
- withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
- writer.write(table)
- }
- buffer
- }
-
- /**
- * This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying 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.
- */
- def gpuConvertCachedBatchToColumnarBatch(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- conf: SQLConf): RDD[ColumnarBatch] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- convertCachedBatchToColumnarInternal(
- input,
- cachedSchemaWithNames,
- selectedSchemaWithNames,
- newSelectedAttributes)
- }
-
- private def convertCachedBatchToColumnarInternal(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- originalSelectedAttributes: Seq[Attribute]): RDD[ColumnarBatch] = {
-
- val cbRdd: RDD[ColumnarBatch] = input.map {
- case parquetCB: ParquetCachedBatch =>
- val parquetOptions = ParquetOptions.builder()
- .includeColumn(selectedAttributes.map(_.name).asJavaCollection).build()
- withResource(Table.readParquet(parquetOptions, parquetCB.buffer, 0,
- parquetCB.sizeInBytes)) { table =>
- withResource {
- for (i <- 0 until table.getNumberOfColumns) yield {
- ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(table.getColumn(i),
- originalSelectedAttributes(i).dataType,
- (dataType, _) => dataType match {
- case d: DecimalType if d.scale < 0 => true
- case _ => false
- },
- (dataType, cv) => {
- //TODO: why do we have to copy to a vector
- dataType match {
- case d: DecimalType =>
- withResource(cv.bitCastTo(DecimalUtil.createCudfDecimal(d))) {
- _.copyToColumnVector()
- }
- case _ =>
- throw new IllegalStateException("We don't cast any type besides Decimal " +
- "with scale < 0")
- }
- }
- )
- }
- } { col =>
- withResource(new Table(col: _*)) { t =>
- GpuColumnVector.from(t, originalSelectedAttributes.map(_.dataType).toArray)
- }
- }
- }
- case _ =>
- throw new IllegalStateException("I don't know how to convert this batch")
- }
- cbRdd
- }
-
- private def getSelectedSchemaFromCachedSchema(
- selectedAttributes: Seq[Attribute],
- cacheAttributes: Seq[Attribute]): Seq[Attribute] = {
- selectedAttributes.map {
- a => cacheAttributes(cacheAttributes.map(_.exprId).indexOf(a.exprId))
- }
- }
-
- /**
- * 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] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val rapidsConf = new RapidsConf(conf)
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- if (rapidsConf.isSqlEnabled &&
- isSchemaSupportedByCudf(cachedSchemaWithNames)) {
- val batches = convertCachedBatchToColumnarInternal(input, cachedSchemaWithNames,
- selectedSchemaWithNames, newSelectedAttributes)
- val cbRdd = batches.map(batch => {
- withResource(batch) { gpuBatch =>
- val cols = GpuColumnVector.extractColumns(gpuBatch)
- new ColumnarBatch(cols.safeMap(_.copyToHost()).toArray, gpuBatch.numRows())
- }
- })
- cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter))
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, newSelectedAttributes, broadcastedConf).getColumnBatchIterator
- }
- }
- }
- }
-
- /**
- * 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 (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes)
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, selectedAttributes, broadcastedConf).getInternalRowIterator
- }
- }
- }
-
- private abstract class UnsupportedDataHandlerIterator extends Iterator[InternalRow] {
-
- def handleInternalRow(schema: Seq[Attribute], row: InternalRow, newRow: InternalRow): Unit
-
- def handleInterval(data: SpecializedGetters, index: Int): Any
-
- def handleStruct(
- data: InternalRow,
- origSchema: StructType,
- supportedSchema: StructType): InternalRow = {
- val structRow = InternalRow.fromSeq(supportedSchema)
- handleInternalRow(origSchema.map(field =>
- AttributeReference(field.name, field.dataType, field.nullable)()), data, structRow)
- structRow
- }
-
- def handleMap(
- keyType: DataType,
- valueType: DataType,
- mapData: MapData): MapData = {
- val keyData = mapData.keyArray()
- val newKeyData = handleArray(keyType, keyData)
- val valueData = mapData.valueArray()
- val newValueData = handleArray(valueType, valueData)
- new ArrayBasedMapData(newKeyData, newValueData)
- }
-
- def handleArray(
- dataType: DataType,
- arrayData: ArrayData): ArrayData = {
- dataType match {
- case s@StructType(_) =>
- val listBuffer = new ListBuffer[InternalRow]()
- val supportedSchema = mapping(dataType).asInstanceOf[StructType]
- arrayData.foreach(supportedSchema, (_, data) => {
- val structRow =
- handleStruct(data.asInstanceOf[InternalRow], s, s)
- listBuffer += structRow.copy()
- })
- new GenericArrayData(listBuffer)
-
- case ArrayType(elementType, _) =>
- val arrayList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val subArrayData = arrayData.getArray(i)
- arrayList.append(handleArray(elementType, subArrayData))
- }
- new GenericArrayData(arrayList)
-
- case m@MapType(_, _, _) =>
- val mapList =
- new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val mapData = arrayData.getMap(i)
- mapList.append(handleMap(m.keyType, m.valueType, mapData))
- }
- new GenericArrayData(mapList)
-
- case CalendarIntervalType =>
- val citList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val citRow = handleInterval(arrayData, i)
- citList += citRow
- }
- new GenericArrayData(citList)
-
- case _ =>
- arrayData
- }
- }
- }
-
- /**
- * Consumes the Iterator[CachedBatch] to return either Iterator[ColumnarBatch] or
- * Iterator[InternalRow]
- */
- private class CachedBatchIteratorConsumer(
- cbIter: Iterator[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- origCacheSchema: Seq[Attribute],
- origRequestedSchema: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val hadoopConf: Configuration = getHadoopConf(origRequestedSchema.toStructType, conf)
- val options: ParquetReadOptions = HadoopReadOptions.builder(hadoopConf).build()
- /**
- * We are getting this method using reflection because its a package-private
- */
- val readBatchMethod: Method =
- classOf[VectorizedColumnReader].getDeclaredMethod("readBatch", Integer.TYPE,
- classOf[WritableColumnVector])
- readBatchMethod.setAccessible(true)
-
- def getInternalRowIterator: Iterator[InternalRow] = {
-
- /**
- * This iterator converts an iterator[CachedBatch] to an iterator[InternalRow].
- *
- * This makes it unlike a regular iterator because CachedBatch => InternalRow* is a 1-n
- * relation. The way we have implemented this is to first go through the
- * iterator[CachedBatch] (cbIter) to look for a valid iterator (iter) i.e. hasNext() => true.
- * Then every time next() is called we return a single InternalRow from iter. When
- * iter.hasNext() => false, we find the next valid iterator in cbIter and the process
- * continues as above.
- */
- new Iterator[InternalRow]() {
-
- var iter: Iterator[InternalRow] = _
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = convertCachedBatchToInternalRowIter
- }
- iter != null && iter.hasNext
- }
-
- override def next(): InternalRow = {
- // will return the next InternalRow if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- /**
- * This method converts a CachedBatch to an iterator of InternalRows.
- */
- private def convertCachedBatchToInternalRowIter: Iterator[InternalRow] = {
- val parquetCachedBatch = cbIter.next().asInstanceOf[ParquetCachedBatch]
- val inputFile = new ByteArrayInputFile(parquetCachedBatch.buffer)
- withResource(ParquetFileReader.open(inputFile, options)) { parquetFileReader =>
- val parquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val hasUnsupportedType = origCacheSchema.exists { field =>
- !isTypeSupportedByParquet(field.dataType)
- }
-
- val unsafeRows = new ArrayBuffer[InternalRow]
- import org.apache.parquet.io.ColumnIOFactory
- var pages = parquetFileReader.readNextRowGroup()
- while (pages != null) {
- val rows = pages.getRowCount
- val columnIO = new ColumnIOFactory().getColumnIO(parquetSchema)
- val recordReader =
- columnIO.getRecordReader(pages, new ParquetRecordMaterializer(parquetSchema,
- cacheAttributes.toStructType,
- new ParquetToSparkSchemaConverter(hadoopConf), None /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED))
- for (_ <- 0 until rows.toInt) {
- val row = recordReader.read
- unsafeRows += row.copy()
- }
- pages = parquetFileReader.readNextRowGroup()
- }
-
- val iter = unsafeRows.iterator
- val unsafeProjection =
- GenerateUnsafeProjection.generate(selectedAttributes, cacheAttributes)
- if (hasUnsupportedType) {
- new UnsupportedDataHandlerIterator() {
- val wrappedIter: Iterator[InternalRow] = iter
- val newRow = new GenericInternalRow(cacheAttributes.length)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- //read a row and convert it to what the caller is expecting
- val row = wrappedIter.next()
- handleInternalRow(origCacheSchema, row, newRow)
- val unsafeProjection =
- GenerateUnsafeProjection.generate(origRequestedSchema, origCacheSchema)
- unsafeProjection.apply(newRow)
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): CalendarInterval = {
- if (data.isNullAt(index)) {
- null
- } else {
- val structData = data.getStruct(index, 3)
- new CalendarInterval(structData.getInt(0),
- structData.getInt(1), structData.getLong(2))
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val supportedSchema = mapping(dataType)
- .asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, supportedSchema.size), s, s)
- newRow.update(index, structRow)
-
- case a@ArrayType(_, _) =>
- val arrayData = row.getArray(index)
- newRow.update(index, handleArray(a.elementType, arrayData))
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- newRow.update(index, handleMap(keyType, valueType, mapData))
-
- case CalendarIntervalType =>
- val interval = handleInterval(row, index)
- if (interval == null) {
- newRow.setNullAt(index)
- } else {
- newRow.setInterval(index, interval)
- }
- case d: DecimalType =>
- if (row.isNullAt(index)) {
- newRow.setDecimal(index, null, d.precision)
- } else {
- val dec = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- Decimal(row.getInt(index).toLong, d.precision, d.scale)
- } else {
- Decimal(row.getLong(index), d.precision, d.scale)
- }
- newRow.update(index, dec)
- }
- case NullType =>
- newRow.setNullAt(index)
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- } else {
- iter.map(unsafeProjection)
- }
- }
- }
- }
- }
-
- private class CurrentBatchIterator(val parquetCachedBatch: ParquetCachedBatch)
- extends Iterator[ColumnarBatch] with AutoCloseable {
-
- val capacity = conf.parquetVectorizedReaderBatchSize
- var columnReaders: Array[VectorizedColumnReader] = _
- val columnVectors: Array[OffHeapColumnVector] =
- OffHeapColumnVector.allocateColumns(capacity, selectedAttributes.toStructType)
- val columnarBatch = new ColumnarBatch(columnVectors
- .asInstanceOf[Array[vectorized.ColumnVector]])
- var rowsReturned: Long = 0L
- var numBatched = 0
- var batchIdx = 0
- var totalCountLoadedSoFar: Long = 0
- val parquetFileReader =
- ParquetFileReader.open(new ByteArrayInputFile(parquetCachedBatch.buffer), options)
- val (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache) = {
- val parquetToSparkSchemaConverter = new ParquetToSparkSchemaConverter(hadoopConf)
- // we are getting parquet schema and then converting it to catalyst schema
- // because catalyst schema that we get from Spark doesn't have the exact schema expected
- // by the columnar parquet reader
- val inMemCacheParquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val inMemCacheSparkSchema = parquetToSparkSchemaConverter.convert(inMemCacheParquetSchema)
-
- val totalRowCount = parquetFileReader.getRowGroups.asScala.map(_.getRowCount).sum
- val sparkToParquetSchemaConverter = new SparkToParquetSchemaConverter(hadoopConf)
- val inMemReqSparkSchema = StructType(selectedAttributes.toStructType.map { field =>
- inMemCacheSparkSchema.fields(inMemCacheSparkSchema.fieldIndex(field.name))
- })
- val inMemReqParquetSchema = sparkToParquetSchemaConverter.convert(inMemReqSparkSchema)
- val columnsRequested: util.List[ColumnDescriptor] = inMemReqParquetSchema.getColumns
-
- val reqSparkSchemaInCacheOrder = StructType(inMemCacheSparkSchema.filter(f =>
- inMemReqSparkSchema.fields.exists(f0 => f0.name.equals(f.name))))
-
- // There could be a case especially in a distributed environment where the requestedSchema
- // and cacheSchema are not in the same order. We need to create a map so we can guarantee
- // that we writing to the correct columnVector
- val cacheSchemaToReqSchemaMap: Map[Int, Int] =
- reqSparkSchemaInCacheOrder.indices.map { index =>
- index -> inMemReqSparkSchema.fields.indexOf(reqSparkSchemaInCacheOrder.fields(index))
- }.toMap
-
- val reqParquetSchemaInCacheOrder =
- sparkToParquetSchemaConverter.convert(reqSparkSchemaInCacheOrder)
-
- // reset spark schema calculated from parquet schema
- hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, inMemReqSparkSchema.json)
- hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, inMemReqSparkSchema.json)
-
- val columnsInCache: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
- val typesInCache: util.List[Type] = reqParquetSchemaInCacheOrder.asGroupType.getFields
- val missingColumns = new Array[Boolean](inMemReqParquetSchema.getFieldCount)
-
- // initialize missingColumns to cover the case where requested column isn't present in the
- // cache, which should never happen but just in case it does
- val paths: util.List[Array[String]] = inMemReqParquetSchema.getPaths
-
- for (i <- 0 until inMemReqParquetSchema.getFieldCount) {
- val t = inMemReqParquetSchema.getFields.get(i)
- if (!t.isPrimitive || t.isRepetition(Type.Repetition.REPEATED)) {
- throw new UnsupportedOperationException("Complex types not supported.")
- }
- val colPath = paths.get(i)
- if (inMemCacheParquetSchema.containsPath(colPath)) {
- val fd = inMemCacheParquetSchema.getColumnDescription(colPath)
- if (!(fd == columnsRequested.get(i))) {
- throw new UnsupportedOperationException("Schema evolution not supported.")
- }
- missingColumns(i) = false
- } else {
- if (columnsRequested.get(i).getMaxDefinitionLevel == 0) {
- // Column is missing in data but the required data is non-nullable.
- // This file is invalid.
- throw new IOException(s"Required column is missing in data file: ${colPath.toList}")
- }
- missingColumns(i) = true
- }
- }
-
- for (i <- missingColumns.indices) {
- if (missingColumns(i)) {
- columnVectors(i).putNulls(0, capacity)
- columnVectors(i).setIsConstant()
- }
- }
-
- (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache)
- }
-
- @throws[IOException]
- def checkEndOfRowGroup(): Unit = {
- if (rowsReturned != totalCountLoadedSoFar) return
- val pages = parquetFileReader.readNextRowGroup
- if (pages == null) {
- throw new IOException("expecting more rows but reached last" +
- " block. Read " + rowsReturned + " out of " + totalRowCount)
- }
- columnReaders = new Array[VectorizedColumnReader](columnsRequested.size)
- for (i <- 0 until columnsRequested.size) {
- if (!missingColumns(i)) {
- columnReaders(i) =
- new VectorizedColumnReader(
- columnsInCache.get(i),
- typesInCache.get(i).getOriginalType,
- pages.getPageReader(columnsInCache.get(i)),
- null /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED.toString,
- LegacyBehaviorPolicy.EXCEPTION.toString)
- }
- }
- totalCountLoadedSoFar += pages.getRowCount
- }
-
- /**
- * Read the next RowGroup and read each column and return the columnarBatch
- */
- def nextBatch: Boolean = {
- for (vector <- columnVectors) {
- vector.reset()
- }
- columnarBatch.setNumRows(0)
- if (rowsReturned >= totalRowCount) return false
- checkEndOfRowGroup()
- val num = Math.min(capacity.toLong, totalCountLoadedSoFar - rowsReturned).toInt
- for (i <- columnReaders.indices) {
- if (columnReaders(i) != null) {
- readBatchMethod.invoke(columnReaders(i), num.asInstanceOf[AnyRef],
- columnVectors(cacheSchemaToReqSchemaMap(i)).asInstanceOf[AnyRef])
- }
- }
- rowsReturned += num
- columnarBatch.setNumRows(num)
- numBatched = num
- batchIdx = 0
- true
- }
-
- override def hasNext: Boolean = rowsReturned < totalRowCount
-
- override def next(): ColumnarBatch = {
- if (nextBatch) {
- // FYI, A very IMPORTANT thing to note is that we are returning the columnar batch
- // as-is i.e. this batch has NullTypes saved as IntegerTypes with null values. The
- // way Spark optimizes the read of NullTypes makes this work without having to rip out
- // the IntegerType column to be replaced by a NullType column. This could change in
- // future and will affect this code.
- columnarBatch
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => {
- close()
- })
-
- override def close(): Unit = {
- parquetFileReader.close()
- }
- }
-
- /**
- * This method returns a ColumnarBatch iterator over a CachedBatch.
- * Each CachedBatch => ColumnarBatch is a 1-1 conversion so its pretty straight forward
- */
- def getColumnBatchIterator: Iterator[ColumnarBatch] = new Iterator[ColumnarBatch] {
- var iter = getIterator
-
- def getIterator: Iterator[ColumnarBatch] = {
- if (!cbIter.hasNext) {
- Iterator.empty
- } else {
- new CurrentBatchIterator(cbIter.next().asInstanceOf[ParquetCachedBatch])
- }
- }
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = getIterator
- }
- iter != null && iter.hasNext
- }
-
- override def next(): ColumnarBatch = {
- // will return the next ColumnarBatch if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
- }
- }
-
- private def getConfFromMap(sharedConf: Broadcast[Map[String, String]]): SQLConf = {
- val conf = new SQLConf()
- sharedConf.value.foreach { case (k, v) => conf.setConfString(k, v) }
- conf
- }
-
- private val intervalStructType = new StructType()
- .add("_days", IntegerType)
- .add("_months", IntegerType)
- .add("_ms", LongType)
-
- def getBytesAllowedPerBatch(conf: SQLConf): Long = {
- val gpuBatchSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf)
- // we are rough estimating 0.5% as meta_data_size. we can do better estimation in future
- val approxMetaDataSizeBytes = gpuBatchSize * 0.5/100
- (gpuBatchSize - approxMetaDataSizeBytes).toLong
- }
-
- /**
- * This is a private helper class to return Iterator to convert InternalRow or ColumnarBatch to
- * CachedBatch. There is no type checking so if the type of T is anything besides InternalRow
- * or ColumnarBatch then the behavior is undefined.
- *
- * @param iter - an iterator over InternalRow or ColumnarBatch
- * @param cachedAttributes - Schema of the cached batch
- * @param sharedConf - SQL conf
- * @tparam T - Strictly either InternalRow or ColumnarBatch
- */
- private[rapids] class CachedBatchIteratorProducer[T](
- iter: Iterator[T],
- cachedAttributes: Seq[Attribute],
- origCachedAttributes: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val hadoopConf: Configuration = getHadoopConf(cachedAttributes.toStructType, conf)
-
- def getInternalRowToCachedBatchIterator: Iterator[CachedBatch] = {
- new InternalRowToCachedBatchIterator
- }
-
- def getColumnarBatchToCachedBatchIterator: Iterator[CachedBatch] = {
- new ColumnarBatchToCachedBatchIterator
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[InternalRow]. This is a n-1
- * relationship. Each partition represents a single parquet file, so we encode it
- * and return the CachedBatch when next is called.
- */
- class InternalRowToCachedBatchIterator extends Iterator[CachedBatch]() {
-
- var parquetOutputFileFormat = new ParquetOutputFileFormat()
-
- // For testing only
- private[rapids] def setParquetOutputFileFormat(p: ParquetOutputFileFormat): Unit = {
- parquetOutputFileFormat = p
- }
-
- // is there a type that spark doesn't support by default in the schema?
- val hasUnsupportedType: Boolean = origCachedAttributes.exists { attribute =>
- !isTypeSupportedByParquet(attribute.dataType)
- }
-
- def getIterator: Iterator[InternalRow] = {
- if (!hasUnsupportedType) {
- iter.asInstanceOf[Iterator[InternalRow]]
- } else {
- new UnsupportedDataHandlerIterator {
-
- val wrappedIter: Iterator[InternalRow] = iter.asInstanceOf[Iterator[InternalRow]]
-
- val newRow: InternalRow = InternalRow.fromSeq(cachedAttributes)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- val row = wrappedIter.next()
- handleInternalRow(origCachedAttributes, row, newRow)
- newRow
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): InternalRow = {
- val citRow = InternalRow(IntegerType, IntegerType, LongType)
- if (data.isNullAt(index)) {
- null
- } else {
- val cit = data.getInterval(index)
- citRow.setInt(0, cit.months)
- citRow.setInt(1, cit.days)
- citRow.setLong(2, cit.microseconds)
- citRow
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val newSchema = mapping(dataType).asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, s.fields.length), s, newSchema)
- newRow.update(index, structRow)
-
- case ArrayType(arrayDataType, _) =>
- val arrayData = row.getArray(index)
- val newArrayData = handleArray(arrayDataType, arrayData)
- newRow.update(index, newArrayData)
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- val map = handleMap(keyType, valueType, mapData)
- newRow.update(index, map)
-
- case CalendarIntervalType =>
- val structData: InternalRow = handleInterval(row, index)
- if (structData == null) {
- newRow.setNullAt(index)
- } else {
- newRow.update(index, structData)
- }
-
- case d: DecimalType if d.scale < 0 =>
- if (d.precision <= Decimal.MAX_INT_DIGITS) {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong.toInt)
- } else {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong)
- }
-
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- }
- }
-
- override def hasNext: Boolean = queue.nonEmpty || iter.hasNext
-
- private val queue = new mutable.Queue[CachedBatch]()
-
- //estimate the size of a row
- val estimatedSize: Int = cachedAttributes.map { attr =>
- attr.dataType.defaultSize
- }.sum
-
- override def next(): CachedBatch = {
- if (queue.isEmpty) {
- // to store a row if we have read it but there is no room in the parquet file to put it
- // we will put it in the next CachedBatch
- var leftOverRow: Option[InternalRow] = None
- val rowIterator = getIterator
- while (rowIterator.hasNext || leftOverRow.nonEmpty) {
- // Each partition will be a single parquet file
- var rows = 0
- // at least a single block
- val stream = new ByteArrayOutputStream(ByteArrayOutputFile.BLOCK_SIZE)
- val outputFile: OutputFile = new ByteArrayOutputFile(stream)
- conf.setConfString(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
- val recordWriter = SQLConf.withExistingConf(conf) {
- parquetOutputFileFormat.getRecordWriter(outputFile, hadoopConf)
- }
- var totalSize = 0
- while ((rowIterator.hasNext || leftOverRow.nonEmpty)
- && totalSize < bytesAllowedPerBatch) {
-
- val row = if (leftOverRow.nonEmpty) {
- val a = leftOverRow.get
- leftOverRow = None // reset value
- a
- } else {
- rowIterator.next()
- }
- totalSize += {
- row match {
- case r: UnsafeRow =>
- r.getSizeInBytes
- case _ =>
- estimatedSize
- }
- }
- if (totalSize <= bytesAllowedPerBatch) {
- rows += 1
- if (rows < 0) {
- throw new IllegalStateException("CachedBatch doesn't support rows larger " +
- "than Int.MaxValue")
- }
- recordWriter.write(null, row)
- } else {
- leftOverRow = Some(if (row.isInstanceOf[UnsafeRow]) {
- row.copy()
- } else {
- row
- })
- }
- }
- // passing null as context isn't used in this method
- recordWriter.close(null)
- queue += ParquetCachedBatch(rows, stream.toByteArray)
- }
- }
- queue.dequeue()
- }
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[ColumnarBatch]. This is a 1-1
- * relationship. Each ColumnarBatch is converted to a single ParquetCachedBatch when next()
- * is called on this iterator
- */
- class ColumnarBatchToCachedBatchIterator extends InternalRowToCachedBatchIterator {
- override def getIterator: Iterator[InternalRow] = {
-
- new Iterator[InternalRow] {
- // We have to check for null context because of the unit test
- Option(TaskContext.get).foreach(_.addTaskCompletionListener[Unit](_ => hostBatch.close()))
-
- val batch: ColumnarBatch = iter.asInstanceOf[Iterator[ColumnarBatch]].next
- val hostBatch = if (batch.column(0).isInstanceOf[GpuColumnVector]) {
- withResource(batch) { batch =>
- new ColumnarBatch(batch.safeMap(_.copyToHost()).toArray, batch.numRows())
- }
- } else {
- batch
- }
-
- val rowIterator = hostBatch.rowIterator().asScala
-
- override def next: InternalRow = rowIterator.next
-
- override def hasNext: Boolean = rowIterator.hasNext
-
- }
- }
- }
-
- }
-
- val mapping = new mutable.HashMap[DataType, DataType]()
-
- def getSupportedDataType(curId: AtomicLong, dataType: DataType): DataType = {
- dataType match {
- case CalendarIntervalType =>
- intervalStructType
- case NullType =>
- ByteType
- case s: StructType =>
- val newStructType = StructType(
- s.indices.map { index =>
- StructField(curId.getAndIncrement().toString,
- getSupportedDataType(curId, s.fields(index).dataType), s.fields(index).nullable,
- s.fields(index).metadata)
- })
- mapping.put(s, newStructType)
- newStructType
- case a@ArrayType(elementType, nullable) =>
- val newArrayType =
- ArrayType(getSupportedDataType(curId, elementType), nullable)
- mapping.put(a, newArrayType)
- newArrayType
- case m@MapType(keyType, valueType, nullable) =>
- val newKeyType = getSupportedDataType(curId, keyType)
- val newValueType = getSupportedDataType(curId, valueType)
- val mapType = MapType(newKeyType, newValueType, nullable)
- mapping.put(m, mapType)
- mapType
- case d: DecimalType if d.scale < 0 =>
- val newType = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- IntegerType
- } else {
- LongType
- }
- newType
- case _ =>
- dataType
- }
- }
-
- private def getSupportedSchemaFromUnsupported(
- cachedAttributes: Seq[Attribute],
- requestedAttributes: Seq[Attribute] = Seq.empty): (Seq[Attribute], Seq[Attribute]) = {
-
- // We only handle CalendarIntervalType, Decimals and NullType ATM convert it to a supported type
- val curId = new AtomicLong()
- val newCachedAttributes = cachedAttributes.map {
- attribute => val name = s"_col${curId.getAndIncrement()}"
- attribute.dataType match {
- case CalendarIntervalType =>
- AttributeReference(name, intervalStructType,
- attribute.nullable, metadata = attribute.metadata)(attribute.exprId)
- .asInstanceOf[Attribute]
- case NullType =>
- AttributeReference(name, DataTypes.ByteType,
- nullable = true, metadata =
- attribute.metadata)(attribute.exprId).asInstanceOf[Attribute]
- case StructType(_) | ArrayType(_, _) | MapType(_, _, _) | DecimalType() =>
- AttributeReference(name,
- getSupportedDataType(curId, attribute.dataType),
- attribute.nullable, attribute.metadata)(attribute.exprId)
- case _ =>
- attribute.withName(name)
- }
- }
-
- val newRequestedAttributes =
- getSelectedSchemaFromCachedSchema(requestedAttributes, newCachedAttributes)
-
- (newCachedAttributes, newRequestedAttributes)
- }
-
- private def getHadoopConf(requestedSchema: StructType,
- sqlConf: SQLConf): Configuration = {
-
- val hadoopConf = new Configuration(false)
- hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
- hadoopConf.set(
- ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- ParquetWriteSupport.SPARK_ROW_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sqlConf.sessionLocalTimeZone)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, false)
- hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, false)
-
- hadoopConf.set(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
-
- hadoopConf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key,
- SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, false)
-
- hadoopConf.set(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString)
-
- ParquetWriteSupport.setSchema(requestedSchema, hadoopConf)
-
- hadoopConf
- }
-
- /**
- * 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- val structSchema = schemaWithUnambiguousNames.toStructType
- val converters = new GpuRowToColumnConverter(structSchema)
- val columnarBatchRdd = input.mapPartitions(iter => {
- new RowToColumnarIterator(iter, structSchema, RequireSingleBatch, converters)
- })
- columnarBatchRdd.flatMap(cb => {
- withResource(cb)(cb => compressColumnarBatchWithParquet(cb, structSchema,
- schema.toStructType, bytesAllowedPerBatch))
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[InternalRow](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getInternalRowToCachedBatchIterator
- }
- }
- }
-
- override def buildFilter(
- predicates: Seq[Expression],
- cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
- //essentially a noop
- (_: Int, b: Iterator[CachedBatch]) => b
- }
-}
-
-/**
- * Similar to ParquetFileFormat
- */
-private[rapids] class ParquetOutputFileFormat {
-
- def getRecordWriter(output: OutputFile, conf: Configuration): RecordWriter[Void, InternalRow] = {
- import ParquetOutputFormat._
-
- val blockSize = getLongBlockSize(conf)
- val maxPaddingSize =
- conf.getInt(MAX_PADDING_BYTES, ParquetWriter.MAX_PADDING_SIZE_DEFAULT)
- val validating = getValidation(conf)
-
- val writeSupport = new ParquetWriteSupport().asInstanceOf[WriteSupport[InternalRow]]
- val init = writeSupport.init(conf)
- val writer = new ParquetFileWriter(output, init.getSchema,
- Mode.CREATE, blockSize, maxPaddingSize)
- writer.start()
-
- val writerVersion =
- ParquetProperties.WriterVersion.fromString(conf.get(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString))
-
- val codecFactory = new CodecFactory(conf, getPageSize(conf))
-
- new ParquetRecordWriter[InternalRow](writer, writeSupport, init.getSchema,
- init.getExtraMetaData, blockSize, getPageSize(conf),
- codecFactory.getCompressor(CompressionCodecName.UNCOMPRESSED), getDictionaryPageSize(conf),
- getEnableDictionary(conf), validating, writerVersion,
- ParquetOutputFileFormat.getMemoryManager(conf))
-
- }
-}
-
-private object ParquetOutputFileFormat {
- var memoryManager: MemoryManager = _
- val DEFAULT_MEMORY_POOL_RATIO: Float = 0.95f
- val DEFAULT_MIN_MEMORY_ALLOCATION: Long = 1 * 1024 * 1024 // 1MB
-
- def getMemoryManager(conf: Configuration): MemoryManager = {
- synchronized {
- if (memoryManager == null) {
- import ParquetOutputFormat._
- val maxLoad = conf.getFloat(MEMORY_POOL_RATIO, DEFAULT_MEMORY_POOL_RATIO)
- val minAllocation = conf.getLong(MIN_MEMORY_ALLOCATION, DEFAULT_MIN_MEMORY_ALLOCATION)
- memoryManager = new MemoryManager(maxLoad, minAllocation)
- }
- }
- memoryManager
- }
-}
diff --git a/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/SparkBaseShims.scala b/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/SparkBaseShims.scala
index d189c8d9898..41e215f70d4 100644
--- a/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/SparkBaseShims.scala
+++ b/shims/spark311db/src/main/scala/com/nvidia/spark/rapids/shims/spark311db/SparkBaseShims.scala
@@ -20,6 +20,7 @@ import java.net.URI
import java.nio.ByteBuffer
import com.databricks.sql.execution.window.RunningWindowFunctionExec
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.spark311db._
import com.nvidia.spark.rapids.shims.v2.Spark30XShims
@@ -59,6 +60,7 @@ import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuB
import org.apache.spark.sql.rapids.execution.python.{GpuAggregateInPandasExecMeta, GpuArrowEvalPythonExec, GpuMapInPandasExecMeta, GpuPythonUDF}
import org.apache.spark.sql.rapids.execution.python.shims.spark311db._
import org.apache.spark.sql.rapids.shims.spark311db._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
diff --git a/shims/spark311db/src/main/scala/org/apache/spark/sql/rapids/shims/spark311db/GpuInMemoryTableScanExec.scala b/shims/spark311db/src/main/scala/org/apache/spark/sql/rapids/shims/spark311db/GpuInMemoryTableScanExec.scala
deleted file mode 100644
index aa018fa85ed..00000000000
--- a/shims/spark311db/src/main/scala/org/apache/spark/sql/rapids/shims/spark311db/GpuInMemoryTableScanExec.scala
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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 org.apache.spark.sql.rapids.shims.spark311db
-
-import com.nvidia.spark.rapids.{GpuExec, GpuMetric}
-import com.nvidia.spark.rapids.shims.spark311db.ParquetCachedBatchSerializer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.physical.Partitioning
-import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
-import org.apache.spark.sql.vectorized.ColumnarBatch
-
-case class GpuInMemoryTableScanExec(
- attributes: Seq[Attribute],
- predicates: Seq[Expression],
- @transient relation: InMemoryRelation) extends LeafExecNode with GpuExec {
-
- override val nodeName: String = {
- relation.cacheBuilder.tableName match {
- case Some(_) =>
- "Scan " + relation.cacheBuilder.cachedName
- case _ =>
- super.nodeName
- }
- }
-
- override def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren
-
- override def doCanonicalize(): SparkPlan =
- copy(attributes = attributes.map(QueryPlan.normalizeExpressions(_, relation.output)),
- predicates = predicates.map(QueryPlan.normalizeExpressions(_, relation.output)),
- relation = relation.canonicalized.asInstanceOf[InMemoryRelation])
-
- override def vectorTypes: Option[Seq[String]] =
- relation.cacheBuilder.serializer.vectorTypes(attributes, conf)
-
- private lazy val columnarInputRDD: RDD[ColumnarBatch] = {
- val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS)
- val buffers = filteredCachedBatches()
- relation.cacheBuilder.serializer.asInstanceOf[ParquetCachedBatchSerializer]
- .gpuConvertCachedBatchToColumnarBatch(
- buffers,
- relation.output,
- attributes,
- conf).map { cb =>
- numOutputRows += cb.numRows()
- cb
- }
- }
-
- override def output: Seq[Attribute] = attributes
-
- private def updateAttribute(expr: Expression): Expression = {
- // attributes can be pruned so using relation's output.
- // E.g., relation.output is [id, item] but this scan's output can be [item] only.
- val attrMap = AttributeMap(relation.cachedPlan.output.zip(relation.output))
- expr.transform {
- case attr: Attribute => attrMap.getOrElse(attr, attr)
- }
- }
-
- // The cached version does not change the outputPartitioning of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputPartitioning: Partitioning = {
- relation.cachedPlan.outputPartitioning match {
- case e: Expression => updateAttribute(e).asInstanceOf[Partitioning]
- case other => other
- }
- }
-
- // The cached version does not change the outputOrdering of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputOrdering: Seq[SortOrder] =
- relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder])
-
- lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled
-
- // Accumulators used for testing purposes
- lazy val readPartitions = sparkContext.longAccumulator
- lazy val readBatches = sparkContext.longAccumulator
-
- private def filteredCachedBatches() = {
- // Right now just return the batch without filtering
- relation.cacheBuilder.cachedColumnBuffers
- }
-
- protected override def doExecute(): RDD[InternalRow] = {
- throw new UnsupportedOperationException("This Exec only deals with Columnar Data")
- }
-
- protected override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- columnarInputRDD
- }
-}
diff --git a/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/ParquetCachedBatchSerializer.scala b/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/ParquetCachedBatchSerializer.scala
deleted file mode 100644
index b793fded0de..00000000000
--- a/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/ParquetCachedBatchSerializer.scala
+++ /dev/null
@@ -1,1521 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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.spark312
-
-import java.io.{InputStream, IOException}
-import java.lang.reflect.Method
-import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-
-import ai.rapids.cudf._
-import ai.rapids.cudf.ParquetWriterOptions.StatisticsFrequency
-import com.nvidia.spark.rapids._
-import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
-import com.nvidia.spark.rapids.RapidsPluginImplicits._
-import java.util
-import org.apache.commons.io.output.ByteArrayOutputStream
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.RecordWriter
-import org.apache.parquet.{HadoopReadOptions, ParquetReadOptions}
-import org.apache.parquet.column.{ColumnDescriptor, ParquetProperties}
-import org.apache.parquet.hadoop.{CodecFactory, MemoryManager, ParquetFileReader, ParquetFileWriter, ParquetInputFormat, ParquetOutputFormat, ParquetRecordWriter, ParquetWriter}
-import org.apache.parquet.hadoop.ParquetFileWriter.Mode
-import org.apache.parquet.hadoop.api.WriteSupport
-import org.apache.parquet.hadoop.metadata.CompressionCodecName
-import org.apache.parquet.io.{DelegatingPositionOutputStream, DelegatingSeekableInputStream, InputFile, OutputFile, PositionOutputStream, SeekableInputStream}
-import org.apache.parquet.schema.{MessageType, Type}
-
-import org.apache.spark.TaskContext
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{vectorized, SparkSession}
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericInternalRow, SpecializedGetters, UnsafeRow}
-import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
-import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
-import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer}
-import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetToSparkSchemaConverter, ParquetWriteSupport, SparkToParquetSchemaConverter, VectorizedColumnReader}
-import org.apache.spark.sql.execution.datasources.parquet.rapids.shims.spark312.ParquetRecordMaterializer
-import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.unsafe.types.CalendarInterval
-
-/**
- * copied from Spark org.apache.spark.util.ByteBufferInputStream
- */
-private class ByteBufferInputStream(private var buffer: ByteBuffer)
- extends InputStream {
-
- override def read(): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- buffer.get() & 0xFF
- }
- }
-
- override def read(dest: Array[Byte]): Int = {
- read(dest, 0, dest.length)
- }
-
- override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- val amountToGet = math.min(buffer.remaining(), length)
- buffer.get(dest, offset, amountToGet)
- amountToGet
- }
- }
-
- override def skip(bytes: Long): Long = {
- if (buffer != null) {
- val amountToSkip = math.min(bytes, buffer.remaining).toInt
- buffer.position(buffer.position() + amountToSkip)
- if (buffer.remaining() == 0) {
- cleanUp()
- }
- amountToSkip
- } else {
- 0L
- }
- }
-
- /**
- * Clean up the buffer, and potentially dispose of it using StorageUtils.dispose().
- */
- private def cleanUp(): Unit = {
- if (buffer != null) {
- buffer = null
- }
- }
-}
-
-class ByteArrayInputFile(buff: Array[Byte]) extends InputFile {
-
- override def getLength: Long = buff.length
-
- override def newStream(): SeekableInputStream = {
- val byteBuffer = ByteBuffer.wrap(buff)
- new DelegatingSeekableInputStream(new ByteBufferInputStream(byteBuffer)) {
- override def getPos: Long = byteBuffer.position()
-
- override def seek(newPos: Long): Unit = {
- if (newPos > Int.MaxValue || newPos < Int.MinValue) {
- throw new IllegalStateException("seek value is out of supported range " + newPos)
- }
- byteBuffer.position(newPos.toInt)
- }
- }
- }
-}
-
-private object ByteArrayOutputFile {
- val BLOCK_SIZE: Int = 32 * 1024 * 1024 // 32M
-}
-
-private class ByteArrayOutputFile(stream: ByteArrayOutputStream) extends OutputFile {
- override def create(blockSizeHint: Long): PositionOutputStream = {
- new DelegatingPositionOutputStream(stream) {
- var pos = 0
-
- override def getPos: Long = pos
-
- override def write(b: Int): Unit = {
- super.write(b)
- pos += Integer.BYTES
- }
-
- override def write(b: Array[Byte]): Unit = {
- super.write(b)
- pos += b.length
- }
-
- override def write(b: Array[Byte], off: Int, len: Int): Unit = {
- super.write(b, off, len)
- pos += len
- }
- }
- }
-
- override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream =
- throw new UnsupportedOperationException("Don't need to overwrite")
-
- override def supportsBlockSize(): Boolean = true
-
- override def defaultBlockSize(): Long = ByteArrayOutputFile.BLOCK_SIZE
-}
-
-private class ParquetBufferConsumer(val numRows: Int) extends HostBufferConsumer with
- AutoCloseable {
- @transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
- private var buffer: Array[Byte] = _
-
- override def handleBuffer(buffer: HostMemoryBuffer, len: Long): Unit = {
- offHeapBuffers += Tuple2(buffer, len)
- }
-
- def getBuffer: Array[Byte] = {
- if (buffer == null) {
- writeBuffers()
- }
- buffer
- }
-
- def close(): Unit = {
- if (buffer == null) {
- writeBuffers()
- }
- }
-
- private def writeBuffers(): Unit = {
- val toProcess = offHeapBuffers.dequeueAll(_ => true)
- // We are making sure the input is smaller than 2gb so the parquet written should never be more
- // than Int.MAX_SIZE.
- val bytes = toProcess.map(_._2).sum
-
- // for now assert bytes are less than Int.MaxValue
- assert(bytes <= Int.MaxValue)
- buffer = new Array(bytes.toInt)
- try {
- var offset: Int = 0
- toProcess.foreach(ops => {
- val origBuffer = ops._1
- val len = ops._2.toInt
- origBuffer.asByteBuffer().get(buffer, offset, len)
- offset = offset + len
- })
- } finally {
- toProcess.map(_._1).safeClose()
- }
- }
-}
-
-private object ParquetCachedBatch {
- def apply(parquetBuff: ParquetBufferConsumer): ParquetCachedBatch = {
- new ParquetCachedBatch(parquetBuff.numRows, parquetBuff.getBuffer)
- }
-}
-
-case class ParquetCachedBatch(
- numRows: Int,
- buffer: Array[Byte]) extends CachedBatch {
- override def sizeInBytes: Long = buffer.length
-}
-
-/**
- * Spark wants the producer to close the batch. We have a listener in this iterator that will close
- * the batch after the task is completed
- */
-private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
- Iterator[ColumnarBatch] {
- var cb: ColumnarBatch = _
-
- private def closeCurrentBatch(): Unit = {
- if (cb != null) {
- cb.close()
- cb = null
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: 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 = schema.fields.forall { f =>
- // only check spark b/c if we are on the GPU then we will be calling the gpu method regardless
- isTypeSupportedByColumnarSparkParquetWriter(f.dataType) || f.dataType == DataTypes.NullType
- }
-
- private def isTypeSupportedByColumnarSparkParquetWriter(dataType: DataType): Boolean = {
- // Columnar writer in Spark only supports AtomicTypes ATM
- dataType match {
- case TimestampType | StringType | BooleanType | DateType | BinaryType |
- DoubleType | FloatType | ByteType | IntegerType | LongType | ShortType => true
- case _: DecimalType => true
- case _ => false
- }
- }
-
- def isSchemaSupportedByCudf(schema: Seq[Attribute]): Boolean = {
- schema.forall(field => isSupportedByCudf(field.dataType))
- }
-
- def isSupportedByCudf(dataType: DataType): Boolean = {
- dataType match {
- // TODO: when arrays are supported for cudf writes add it here.
- // https://github.com/NVIDIA/spark-rapids/issues/2054
- case s: StructType => s.forall(field => isSupportedByCudf(field.dataType))
- case _ => GpuColumnVector.isNonNestedSupportedType(dataType)
- }
- }
-
- /**
- * This method checks if the datatype passed is officially supported by parquet.
- *
- * Please refer to https://github.com/apache/parquet-format/blob/master/LogicalTypes.md to see
- * the what types are supported by parquet
- */
- def isTypeSupportedByParquet(dataType: DataType): Boolean = {
- dataType match {
- case CalendarIntervalType | NullType => false
- case s: StructType => s.forall(field => isTypeSupportedByParquet(field.dataType))
- case ArrayType(elementType, _) => isTypeSupportedByParquet(elementType)
- case MapType(keyType, valueType, _) => isTypeSupportedByParquet(keyType) &&
- isTypeSupportedByParquet(valueType)
- case d: DecimalType if d.scale < 0 => false
- case _ => 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- val structSchema = schemaWithUnambiguousNames.toStructType
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- def putOnGpuIfNeeded(batch: ColumnarBatch): ColumnarBatch = {
- if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
- val s: StructType = structSchema
- val gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows()).build(batch.numRows())
- batch.close()
- gpuCB
- } else {
- batch
- }
- }
-
- input.flatMap(batch => {
- if (batch.numCols() == 0) {
- List(ParquetCachedBatch(batch.numRows(), new Array[Byte](0)))
- } else {
- withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
- compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
- bytesAllowedPerBatch)
- }
- }
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[ColumnarBatch](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getColumnarBatchToCachedBatchIterator
- }
- }
- }
-
- private[rapids] def compressColumnarBatchWithParquet(
- oldGpuCB: ColumnarBatch,
- schema: StructType,
- origSchema: StructType,
- bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
- val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
- oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
- .getBase.getDeviceMemorySize / oldGpuCB.numRows()
- }.sum
-
- val columns = for (i <- 0 until oldGpuCB.numCols()) yield {
- val gpuVector = oldGpuCB.column(i).asInstanceOf[GpuColumnVector]
- var dataType = origSchema(i).dataType
- val v = ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(gpuVector.getBase,
- origSchema(i).dataType,
- // we are checking for scale > 0 because cudf and spark refer to scales as opposites
- // e.g. scale = -3 in Spark is scale = 3 in cudf
- (_, cv) => cv.getType.isDecimalType && cv.getType.getScale > 0,
- (_, cv) => {
- if (cv.getType.isBackedByLong) {
- dataType = LongType
- cv.bitCastTo(DType.INT64)
- } else {
- dataType = IntegerType
- cv.bitCastTo(DType.INT32)
- }
- }
- )
- GpuColumnVector.from(v, schema(i).dataType)
- }
- withResource(new ColumnarBatch(columns.toArray, oldGpuCB.numRows())) { gpuCB =>
- val rowsAllowedInBatch = (bytesAllowedPerBatch / estimatedRowSize).toInt
- val splitIndices = scala.Range(rowsAllowedInBatch, gpuCB.numRows(), rowsAllowedInBatch)
- val buffers = new ListBuffer[ParquetCachedBatch]
- if (splitIndices.nonEmpty) {
- val splitVectors = new ListBuffer[Array[ColumnVector]]
- try {
- for (index <- 0 until gpuCB.numCols()) {
- splitVectors +=
- gpuCB.column(index).asInstanceOf[GpuColumnVector].getBase.split(splitIndices: _*)
- }
-
- // Splitting the table
- // e.g. T0 = {col1, col2,...,coln} => split columns into 'm' cols =>
- // T00= {splitCol1(0), splitCol2(0),...,splitColn(0)}
- // T01= {splitCol1(1), splitCol2(1),...,splitColn(1)}
- // ...
- // T0m= {splitCol1(m), splitCol2(m),...,splitColn(m)}
- def makeTableForIndex(i: Int): Table = {
- val columns = splitVectors.indices.map(j => splitVectors(j)(i))
- new Table(columns: _*)
- }
-
- for (i <- splitVectors.head.indices) {
- withResource(makeTableForIndex(i)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- } finally {
- splitVectors.foreach(array => array.safeClose())
- }
- } else {
- withResource(GpuColumnVector.from(gpuCB)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- buffers.toList
- }
- }
-
- private def writeTableToCachedBatch(
- table: Table,
- schema: StructType): ParquetBufferConsumer = {
- val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
- val opts = GpuParquetFileFormat
- .parquetWriterOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
- .withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
- withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
- writer.write(table)
- }
- buffer
- }
-
- /**
- * This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying 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.
- */
- def gpuConvertCachedBatchToColumnarBatch(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- conf: SQLConf): RDD[ColumnarBatch] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- convertCachedBatchToColumnarInternal(
- input,
- cachedSchemaWithNames,
- selectedSchemaWithNames,
- newSelectedAttributes)
- }
-
- private def convertCachedBatchToColumnarInternal(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- originalSelectedAttributes: Seq[Attribute]): RDD[ColumnarBatch] = {
-
- val cbRdd: RDD[ColumnarBatch] = input.map {
- case parquetCB: ParquetCachedBatch =>
- val parquetOptions = ParquetOptions.builder()
- .includeColumn(selectedAttributes.map(_.name).asJavaCollection).build()
- withResource(Table.readParquet(parquetOptions, parquetCB.buffer, 0,
- parquetCB.sizeInBytes)) { table =>
- withResource {
- for (i <- 0 until table.getNumberOfColumns) yield {
- ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(table.getColumn(i),
- originalSelectedAttributes(i).dataType,
- (dataType, _) => dataType match {
- case d: DecimalType if d.scale < 0 => true
- case _ => false
- },
- (dataType, cv) => {
- //TODO: why do we have to copy to a vector
- dataType match {
- case d: DecimalType =>
- withResource(cv.bitCastTo(DecimalUtil.createCudfDecimal(d))) {
- _.copyToColumnVector()
- }
- case _ =>
- throw new IllegalStateException("We don't cast any type besides Decimal " +
- "with scale < 0")
- }
- }
- )
- }
- } { col =>
- withResource(new Table(col: _*)) { t =>
- GpuColumnVector.from(t, originalSelectedAttributes.map(_.dataType).toArray)
- }
- }
- }
- case _ =>
- throw new IllegalStateException("I don't know how to convert this batch")
- }
- cbRdd
- }
-
- private def getSelectedSchemaFromCachedSchema(
- selectedAttributes: Seq[Attribute],
- cacheAttributes: Seq[Attribute]): Seq[Attribute] = {
- selectedAttributes.map {
- a => cacheAttributes(cacheAttributes.map(_.exprId).indexOf(a.exprId))
- }
- }
-
- /**
- * 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] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val rapidsConf = new RapidsConf(conf)
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- if (rapidsConf.isSqlEnabled &&
- isSchemaSupportedByCudf(cachedSchemaWithNames)) {
- val batches = convertCachedBatchToColumnarInternal(input, cachedSchemaWithNames,
- selectedSchemaWithNames, newSelectedAttributes)
- val cbRdd = batches.map(batch => {
- withResource(batch) { gpuBatch =>
- val cols = GpuColumnVector.extractColumns(gpuBatch)
- new ColumnarBatch(cols.safeMap(_.copyToHost()).toArray, gpuBatch.numRows())
- }
- })
- cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter))
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, newSelectedAttributes, broadcastedConf).getColumnBatchIterator
- }
- }
- }
- }
-
- /**
- * 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 (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes)
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, selectedAttributes, broadcastedConf).getInternalRowIterator
- }
- }
- }
-
- private abstract class UnsupportedDataHandlerIterator extends Iterator[InternalRow] {
-
- def handleInternalRow(schema: Seq[Attribute], row: InternalRow, newRow: InternalRow): Unit
-
- def handleInterval(data: SpecializedGetters, index: Int): Any
-
- def handleStruct(
- data: InternalRow,
- origSchema: StructType,
- supportedSchema: StructType): InternalRow = {
- val structRow = InternalRow.fromSeq(supportedSchema)
- handleInternalRow(origSchema.map(field =>
- AttributeReference(field.name, field.dataType, field.nullable)()), data, structRow)
- structRow
- }
-
- def handleMap(
- keyType: DataType,
- valueType: DataType,
- mapData: MapData): MapData = {
- val keyData = mapData.keyArray()
- val newKeyData = handleArray(keyType, keyData)
- val valueData = mapData.valueArray()
- val newValueData = handleArray(valueType, valueData)
- new ArrayBasedMapData(newKeyData, newValueData)
- }
-
- def handleArray(
- dataType: DataType,
- arrayData: ArrayData): ArrayData = {
- dataType match {
- case s@StructType(_) =>
- val listBuffer = new ListBuffer[InternalRow]()
- val supportedSchema = mapping(dataType).asInstanceOf[StructType]
- arrayData.foreach(supportedSchema, (_, data) => {
- val structRow =
- handleStruct(data.asInstanceOf[InternalRow], s, s)
- listBuffer += structRow.copy()
- })
- new GenericArrayData(listBuffer)
-
- case ArrayType(elementType, _) =>
- val arrayList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val subArrayData = arrayData.getArray(i)
- arrayList.append(handleArray(elementType, subArrayData))
- }
- new GenericArrayData(arrayList)
-
- case m@MapType(_, _, _) =>
- val mapList =
- new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val mapData = arrayData.getMap(i)
- mapList.append(handleMap(m.keyType, m.valueType, mapData))
- }
- new GenericArrayData(mapList)
-
- case CalendarIntervalType =>
- val citList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val citRow = handleInterval(arrayData, i)
- citList += citRow
- }
- new GenericArrayData(citList)
-
- case _ =>
- arrayData
- }
- }
- }
-
- /**
- * Consumes the Iterator[CachedBatch] to return either Iterator[ColumnarBatch] or
- * Iterator[InternalRow]
- */
- private class CachedBatchIteratorConsumer(
- cbIter: Iterator[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- origCacheSchema: Seq[Attribute],
- origRequestedSchema: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val hadoopConf: Configuration = getHadoopConf(origRequestedSchema.toStructType, conf)
- val options: ParquetReadOptions = HadoopReadOptions.builder(hadoopConf).build()
- /**
- * We are getting this method using reflection because its a package-private
- */
- val readBatchMethod: Method =
- classOf[VectorizedColumnReader].getDeclaredMethod("readBatch", Integer.TYPE,
- classOf[WritableColumnVector])
- readBatchMethod.setAccessible(true)
-
- def getInternalRowIterator: Iterator[InternalRow] = {
-
- /**
- * This iterator converts an iterator[CachedBatch] to an iterator[InternalRow].
- *
- * This makes it unlike a regular iterator because CachedBatch => InternalRow* is a 1-n
- * relation. The way we have implemented this is to first go through the
- * iterator[CachedBatch] (cbIter) to look for a valid iterator (iter) i.e. hasNext() => true.
- * Then every time next() is called we return a single InternalRow from iter. When
- * iter.hasNext() => false, we find the next valid iterator in cbIter and the process
- * continues as above.
- */
- new Iterator[InternalRow]() {
-
- var iter: Iterator[InternalRow] = _
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = convertCachedBatchToInternalRowIter
- }
- iter != null && iter.hasNext
- }
-
- override def next(): InternalRow = {
- // will return the next InternalRow if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- /**
- * This method converts a CachedBatch to an iterator of InternalRows.
- */
- private def convertCachedBatchToInternalRowIter: Iterator[InternalRow] = {
- val parquetCachedBatch = cbIter.next().asInstanceOf[ParquetCachedBatch]
- val inputFile = new ByteArrayInputFile(parquetCachedBatch.buffer)
- withResource(ParquetFileReader.open(inputFile, options)) { parquetFileReader =>
- val parquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val hasUnsupportedType = origCacheSchema.exists { field =>
- !isTypeSupportedByParquet(field.dataType)
- }
-
- val unsafeRows = new ArrayBuffer[InternalRow]
- import org.apache.parquet.io.ColumnIOFactory
- var pages = parquetFileReader.readNextRowGroup()
- while (pages != null) {
- val rows = pages.getRowCount
- val columnIO = new ColumnIOFactory().getColumnIO(parquetSchema)
- val recordReader =
- columnIO.getRecordReader(pages, new ParquetRecordMaterializer(parquetSchema,
- cacheAttributes.toStructType,
- new ParquetToSparkSchemaConverter(hadoopConf), None /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED))
- for (_ <- 0 until rows.toInt) {
- val row = recordReader.read
- unsafeRows += row.copy()
- }
- pages = parquetFileReader.readNextRowGroup()
- }
-
- val iter = unsafeRows.iterator
- val unsafeProjection =
- GenerateUnsafeProjection.generate(selectedAttributes, cacheAttributes)
- if (hasUnsupportedType) {
- new UnsupportedDataHandlerIterator() {
- val wrappedIter: Iterator[InternalRow] = iter
- val newRow = new GenericInternalRow(cacheAttributes.length)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- //read a row and convert it to what the caller is expecting
- val row = wrappedIter.next()
- handleInternalRow(origCacheSchema, row, newRow)
- val unsafeProjection =
- GenerateUnsafeProjection.generate(origRequestedSchema, origCacheSchema)
- unsafeProjection.apply(newRow)
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): CalendarInterval = {
- if (data.isNullAt(index)) {
- null
- } else {
- val structData = data.getStruct(index, 3)
- new CalendarInterval(structData.getInt(0),
- structData.getInt(1), structData.getLong(2))
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val supportedSchema = mapping(dataType)
- .asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, supportedSchema.size), s, s)
- newRow.update(index, structRow)
-
- case a@ArrayType(_, _) =>
- val arrayData = row.getArray(index)
- newRow.update(index, handleArray(a.elementType, arrayData))
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- newRow.update(index, handleMap(keyType, valueType, mapData))
-
- case CalendarIntervalType =>
- val interval = handleInterval(row, index)
- if (interval == null) {
- newRow.setNullAt(index)
- } else {
- newRow.setInterval(index, interval)
- }
- case d: DecimalType =>
- if (row.isNullAt(index)) {
- newRow.setDecimal(index, null, d.precision)
- } else {
- val dec = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- Decimal(row.getInt(index).toLong, d.precision, d.scale)
- } else {
- Decimal(row.getLong(index), d.precision, d.scale)
- }
- newRow.update(index, dec)
- }
- case NullType =>
- newRow.setNullAt(index)
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- } else {
- iter.map(unsafeProjection)
- }
- }
- }
- }
- }
-
- private class CurrentBatchIterator(val parquetCachedBatch: ParquetCachedBatch)
- extends Iterator[ColumnarBatch] with AutoCloseable {
-
- val capacity = conf.parquetVectorizedReaderBatchSize
- var columnReaders: Array[VectorizedColumnReader] = _
- val columnVectors: Array[OffHeapColumnVector] =
- OffHeapColumnVector.allocateColumns(capacity, selectedAttributes.toStructType)
- val columnarBatch = new ColumnarBatch(columnVectors
- .asInstanceOf[Array[vectorized.ColumnVector]])
- var rowsReturned: Long = 0L
- var numBatched = 0
- var batchIdx = 0
- var totalCountLoadedSoFar: Long = 0
- val parquetFileReader =
- ParquetFileReader.open(new ByteArrayInputFile(parquetCachedBatch.buffer), options)
- val (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache) = {
- val parquetToSparkSchemaConverter = new ParquetToSparkSchemaConverter(hadoopConf)
- // we are getting parquet schema and then converting it to catalyst schema
- // because catalyst schema that we get from Spark doesn't have the exact schema expected
- // by the columnar parquet reader
- val inMemCacheParquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val inMemCacheSparkSchema = parquetToSparkSchemaConverter.convert(inMemCacheParquetSchema)
-
- val totalRowCount = parquetFileReader.getRowGroups.asScala.map(_.getRowCount).sum
- val sparkToParquetSchemaConverter = new SparkToParquetSchemaConverter(hadoopConf)
- val inMemReqSparkSchema = StructType(selectedAttributes.toStructType.map { field =>
- inMemCacheSparkSchema.fields(inMemCacheSparkSchema.fieldIndex(field.name))
- })
- val inMemReqParquetSchema = sparkToParquetSchemaConverter.convert(inMemReqSparkSchema)
- val columnsRequested: util.List[ColumnDescriptor] = inMemReqParquetSchema.getColumns
-
- val reqSparkSchemaInCacheOrder = StructType(inMemCacheSparkSchema.filter(f =>
- inMemReqSparkSchema.fields.exists(f0 => f0.name.equals(f.name))))
-
- // There could be a case especially in a distributed environment where the requestedSchema
- // and cacheSchema are not in the same order. We need to create a map so we can guarantee
- // that we writing to the correct columnVector
- val cacheSchemaToReqSchemaMap: Map[Int, Int] =
- reqSparkSchemaInCacheOrder.indices.map { index =>
- index -> inMemReqSparkSchema.fields.indexOf(reqSparkSchemaInCacheOrder.fields(index))
- }.toMap
-
- val reqParquetSchemaInCacheOrder =
- sparkToParquetSchemaConverter.convert(reqSparkSchemaInCacheOrder)
-
- // reset spark schema calculated from parquet schema
- hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, inMemReqSparkSchema.json)
- hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, inMemReqSparkSchema.json)
-
- val columnsInCache: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
- val typesInCache: util.List[Type] = reqParquetSchemaInCacheOrder.asGroupType.getFields
- val missingColumns = new Array[Boolean](inMemReqParquetSchema.getFieldCount)
-
- // initialize missingColumns to cover the case where requested column isn't present in the
- // cache, which should never happen but just in case it does
- val paths: util.List[Array[String]] = inMemReqParquetSchema.getPaths
-
- for (i <- 0 until inMemReqParquetSchema.getFieldCount) {
- val t = inMemReqParquetSchema.getFields.get(i)
- if (!t.isPrimitive || t.isRepetition(Type.Repetition.REPEATED)) {
- throw new UnsupportedOperationException("Complex types not supported.")
- }
- val colPath = paths.get(i)
- if (inMemCacheParquetSchema.containsPath(colPath)) {
- val fd = inMemCacheParquetSchema.getColumnDescription(colPath)
- if (!(fd == columnsRequested.get(i))) {
- throw new UnsupportedOperationException("Schema evolution not supported.")
- }
- missingColumns(i) = false
- } else {
- if (columnsRequested.get(i).getMaxDefinitionLevel == 0) {
- // Column is missing in data but the required data is non-nullable.
- // This file is invalid.
- throw new IOException(s"Required column is missing in data file: ${colPath.toList}")
- }
- missingColumns(i) = true
- }
- }
-
- for (i <- missingColumns.indices) {
- if (missingColumns(i)) {
- columnVectors(i).putNulls(0, capacity)
- columnVectors(i).setIsConstant()
- }
- }
-
- (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache)
- }
-
- @throws[IOException]
- def checkEndOfRowGroup(): Unit = {
- if (rowsReturned != totalCountLoadedSoFar) return
- val pages = parquetFileReader.readNextRowGroup
- if (pages == null) {
- throw new IOException("expecting more rows but reached last" +
- " block. Read " + rowsReturned + " out of " + totalRowCount)
- }
- columnReaders = new Array[VectorizedColumnReader](columnsRequested.size)
- for (i <- 0 until columnsRequested.size) {
- if (!missingColumns(i)) {
- columnReaders(i) =
- new VectorizedColumnReader(
- columnsInCache.get(i),
- typesInCache.get(i).getOriginalType,
- pages.getPageReader(columnsInCache.get(i)),
- null /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED.toString,
- LegacyBehaviorPolicy.EXCEPTION.toString)
- }
- }
- totalCountLoadedSoFar += pages.getRowCount
- }
-
- /**
- * Read the next RowGroup and read each column and return the columnarBatch
- */
- def nextBatch: Boolean = {
- for (vector <- columnVectors) {
- vector.reset()
- }
- columnarBatch.setNumRows(0)
- if (rowsReturned >= totalRowCount) return false
- checkEndOfRowGroup()
- val num = Math.min(capacity.toLong, totalCountLoadedSoFar - rowsReturned).toInt
- for (i <- columnReaders.indices) {
- if (columnReaders(i) != null) {
- readBatchMethod.invoke(columnReaders(i), num.asInstanceOf[AnyRef],
- columnVectors(cacheSchemaToReqSchemaMap(i)).asInstanceOf[AnyRef])
- }
- }
- rowsReturned += num
- columnarBatch.setNumRows(num)
- numBatched = num
- batchIdx = 0
- true
- }
-
- override def hasNext: Boolean = rowsReturned < totalRowCount
-
- override def next(): ColumnarBatch = {
- if (nextBatch) {
- // FYI, A very IMPORTANT thing to note is that we are returning the columnar batch
- // as-is i.e. this batch has NullTypes saved as IntegerTypes with null values. The
- // way Spark optimizes the read of NullTypes makes this work without having to rip out
- // the IntegerType column to be replaced by a NullType column. This could change in
- // future and will affect this code.
- columnarBatch
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => {
- close()
- })
-
- override def close(): Unit = {
- parquetFileReader.close()
- }
- }
-
- /**
- * This method returns a ColumnarBatch iterator over a CachedBatch.
- * Each CachedBatch => ColumnarBatch is a 1-1 conversion so its pretty straight forward
- */
- def getColumnBatchIterator: Iterator[ColumnarBatch] = new Iterator[ColumnarBatch] {
- var iter = getIterator
-
- def getIterator: Iterator[ColumnarBatch] = {
- if (!cbIter.hasNext) {
- Iterator.empty
- } else {
- new CurrentBatchIterator(cbIter.next().asInstanceOf[ParquetCachedBatch])
- }
- }
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = getIterator
- }
- iter != null && iter.hasNext
- }
-
- override def next(): ColumnarBatch = {
- // will return the next ColumnarBatch if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
- }
- }
-
- private def getConfFromMap(sharedConf: Broadcast[Map[String, String]]): SQLConf = {
- val conf = new SQLConf()
- sharedConf.value.foreach { case (k, v) => conf.setConfString(k, v) }
- conf
- }
-
- private val intervalStructType = new StructType()
- .add("_days", IntegerType)
- .add("_months", IntegerType)
- .add("_ms", LongType)
-
- def getBytesAllowedPerBatch(conf: SQLConf): Long = {
- val gpuBatchSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf)
- // we are rough estimating 0.5% as meta_data_size. we can do better estimation in future
- val approxMetaDataSizeBytes = gpuBatchSize * 0.5/100
- (gpuBatchSize - approxMetaDataSizeBytes).toLong
- }
-
- /**
- * This is a private helper class to return Iterator to convert InternalRow or ColumnarBatch to
- * CachedBatch. There is no type checking so if the type of T is anything besides InternalRow
- * or ColumnarBatch then the behavior is undefined.
- *
- * @param iter - an iterator over InternalRow or ColumnarBatch
- * @param cachedAttributes - Schema of the cached batch
- * @param sharedConf - SQL conf
- * @tparam T - Strictly either InternalRow or ColumnarBatch
- */
- private[rapids] class CachedBatchIteratorProducer[T](
- iter: Iterator[T],
- cachedAttributes: Seq[Attribute],
- origCachedAttributes: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val hadoopConf: Configuration = getHadoopConf(cachedAttributes.toStructType, conf)
-
- def getInternalRowToCachedBatchIterator: Iterator[CachedBatch] = {
- new InternalRowToCachedBatchIterator
- }
-
- def getColumnarBatchToCachedBatchIterator: Iterator[CachedBatch] = {
- new ColumnarBatchToCachedBatchIterator
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[InternalRow]. This is a n-1
- * relationship. Each partition represents a single parquet file, so we encode it
- * and return the CachedBatch when next is called.
- */
- class InternalRowToCachedBatchIterator extends Iterator[CachedBatch]() {
-
- var parquetOutputFileFormat = new ParquetOutputFileFormat()
-
- // For testing only
- private[rapids] def setParquetOutputFileFormat(p: ParquetOutputFileFormat): Unit = {
- parquetOutputFileFormat = p
- }
-
- // is there a type that spark doesn't support by default in the schema?
- val hasUnsupportedType: Boolean = origCachedAttributes.exists { attribute =>
- !isTypeSupportedByParquet(attribute.dataType)
- }
-
- def getIterator: Iterator[InternalRow] = {
- if (!hasUnsupportedType) {
- iter.asInstanceOf[Iterator[InternalRow]]
- } else {
- new UnsupportedDataHandlerIterator {
-
- val wrappedIter: Iterator[InternalRow] = iter.asInstanceOf[Iterator[InternalRow]]
-
- val newRow: InternalRow = InternalRow.fromSeq(cachedAttributes)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- val row = wrappedIter.next()
- handleInternalRow(origCachedAttributes, row, newRow)
- newRow
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): InternalRow = {
- val citRow = InternalRow(IntegerType, IntegerType, LongType)
- if (data.isNullAt(index)) {
- null
- } else {
- val cit = data.getInterval(index)
- citRow.setInt(0, cit.months)
- citRow.setInt(1, cit.days)
- citRow.setLong(2, cit.microseconds)
- citRow
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val newSchema = mapping(dataType).asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, s.fields.length), s, newSchema)
- newRow.update(index, structRow)
-
- case ArrayType(arrayDataType, _) =>
- val arrayData = row.getArray(index)
- val newArrayData = handleArray(arrayDataType, arrayData)
- newRow.update(index, newArrayData)
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- val map = handleMap(keyType, valueType, mapData)
- newRow.update(index, map)
-
- case CalendarIntervalType =>
- val structData: InternalRow = handleInterval(row, index)
- if (structData == null) {
- newRow.setNullAt(index)
- } else {
- newRow.update(index, structData)
- }
-
- case d: DecimalType if d.scale < 0 =>
- if (d.precision <= Decimal.MAX_INT_DIGITS) {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong.toInt)
- } else {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong)
- }
-
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- }
- }
-
- override def hasNext: Boolean = queue.nonEmpty || iter.hasNext
-
- private val queue = new mutable.Queue[CachedBatch]()
-
- //estimate the size of a row
- val estimatedSize: Int = cachedAttributes.map { attr =>
- attr.dataType.defaultSize
- }.sum
-
- override def next(): CachedBatch = {
- if (queue.isEmpty) {
- // to store a row if we have read it but there is no room in the parquet file to put it
- // we will put it in the next CachedBatch
- var leftOverRow: Option[InternalRow] = None
- val rowIterator = getIterator
- while (rowIterator.hasNext || leftOverRow.nonEmpty) {
- // Each partition will be a single parquet file
- var rows = 0
- // at least a single block
- val stream = new ByteArrayOutputStream(ByteArrayOutputFile.BLOCK_SIZE)
- val outputFile: OutputFile = new ByteArrayOutputFile(stream)
- conf.setConfString(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
- val recordWriter = SQLConf.withExistingConf(conf) {
- parquetOutputFileFormat.getRecordWriter(outputFile, hadoopConf)
- }
- var totalSize = 0
- while ((rowIterator.hasNext || leftOverRow.nonEmpty)
- && totalSize < bytesAllowedPerBatch) {
-
- val row = if (leftOverRow.nonEmpty) {
- val a = leftOverRow.get
- leftOverRow = None // reset value
- a
- } else {
- rowIterator.next()
- }
- totalSize += {
- row match {
- case r: UnsafeRow =>
- r.getSizeInBytes
- case _ =>
- estimatedSize
- }
- }
- if (totalSize <= bytesAllowedPerBatch) {
- rows += 1
- if (rows < 0) {
- throw new IllegalStateException("CachedBatch doesn't support rows larger " +
- "than Int.MaxValue")
- }
- recordWriter.write(null, row)
- } else {
- leftOverRow = Some(if (row.isInstanceOf[UnsafeRow]) {
- row.copy()
- } else {
- row
- })
- }
- }
- // passing null as context isn't used in this method
- recordWriter.close(null)
- queue += ParquetCachedBatch(rows, stream.toByteArray)
- }
- }
- queue.dequeue()
- }
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[ColumnarBatch]. This is a 1-1
- * relationship. Each ColumnarBatch is converted to a single ParquetCachedBatch when next()
- * is called on this iterator
- */
- class ColumnarBatchToCachedBatchIterator extends InternalRowToCachedBatchIterator {
- override def getIterator: Iterator[InternalRow] = {
-
- new Iterator[InternalRow] {
- // We have to check for null context because of the unit test
- Option(TaskContext.get).foreach(_.addTaskCompletionListener[Unit](_ => hostBatch.close()))
-
- val batch: ColumnarBatch = iter.asInstanceOf[Iterator[ColumnarBatch]].next
- val hostBatch = if (batch.column(0).isInstanceOf[GpuColumnVector]) {
- withResource(batch) { batch =>
- new ColumnarBatch(batch.safeMap(_.copyToHost()).toArray, batch.numRows())
- }
- } else {
- batch
- }
-
- val rowIterator = hostBatch.rowIterator().asScala
-
- override def next: InternalRow = rowIterator.next
-
- override def hasNext: Boolean = rowIterator.hasNext
-
- }
- }
- }
-
- }
-
- val mapping = new mutable.HashMap[DataType, DataType]()
-
- def getSupportedDataType(curId: AtomicLong, dataType: DataType): DataType = {
- dataType match {
- case CalendarIntervalType =>
- intervalStructType
- case NullType =>
- ByteType
- case s: StructType =>
- val newStructType = StructType(
- s.indices.map { index =>
- StructField(curId.getAndIncrement().toString,
- getSupportedDataType(curId, s.fields(index).dataType), s.fields(index).nullable,
- s.fields(index).metadata)
- })
- mapping.put(s, newStructType)
- newStructType
- case a@ArrayType(elementType, nullable) =>
- val newArrayType =
- ArrayType(getSupportedDataType(curId, elementType), nullable)
- mapping.put(a, newArrayType)
- newArrayType
- case m@MapType(keyType, valueType, nullable) =>
- val newKeyType = getSupportedDataType(curId, keyType)
- val newValueType = getSupportedDataType(curId, valueType)
- val mapType = MapType(newKeyType, newValueType, nullable)
- mapping.put(m, mapType)
- mapType
- case d: DecimalType if d.scale < 0 =>
- val newType = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- IntegerType
- } else {
- LongType
- }
- newType
- case _ =>
- dataType
- }
- }
-
- private def getSupportedSchemaFromUnsupported(
- cachedAttributes: Seq[Attribute],
- requestedAttributes: Seq[Attribute] = Seq.empty): (Seq[Attribute], Seq[Attribute]) = {
-
- // We only handle CalendarIntervalType, Decimals and NullType ATM convert it to a supported type
- val curId = new AtomicLong()
- val newCachedAttributes = cachedAttributes.map {
- attribute => val name = s"_col${curId.getAndIncrement()}"
- attribute.dataType match {
- case CalendarIntervalType =>
- AttributeReference(name, intervalStructType,
- attribute.nullable, metadata = attribute.metadata)(attribute.exprId)
- .asInstanceOf[Attribute]
- case NullType =>
- AttributeReference(name, DataTypes.ByteType,
- nullable = true, metadata =
- attribute.metadata)(attribute.exprId).asInstanceOf[Attribute]
- case StructType(_) | ArrayType(_, _) | MapType(_, _, _) | DecimalType() =>
- AttributeReference(name,
- getSupportedDataType(curId, attribute.dataType),
- attribute.nullable, attribute.metadata)(attribute.exprId)
- case _ =>
- attribute.withName(name)
- }
- }
-
- val newRequestedAttributes =
- getSelectedSchemaFromCachedSchema(requestedAttributes, newCachedAttributes)
-
- (newCachedAttributes, newRequestedAttributes)
- }
-
- private def getHadoopConf(requestedSchema: StructType,
- sqlConf: SQLConf): Configuration = {
-
- val hadoopConf = new Configuration(false)
- hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
- hadoopConf.set(
- ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- ParquetWriteSupport.SPARK_ROW_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sqlConf.sessionLocalTimeZone)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, false)
- hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, false)
-
- hadoopConf.set(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
-
- hadoopConf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key,
- SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, false)
-
- hadoopConf.set(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString)
-
- ParquetWriteSupport.setSchema(requestedSchema, hadoopConf)
-
- hadoopConf
- }
-
- /**
- * 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- val structSchema = schemaWithUnambiguousNames.toStructType
- val converters = new GpuRowToColumnConverter(structSchema)
- val columnarBatchRdd = input.mapPartitions(iter => {
- new RowToColumnarIterator(iter, structSchema, RequireSingleBatch, converters)
- })
- columnarBatchRdd.flatMap(cb => {
- withResource(cb)(cb => compressColumnarBatchWithParquet(cb, structSchema,
- schema.toStructType, bytesAllowedPerBatch))
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[InternalRow](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getInternalRowToCachedBatchIterator
- }
- }
- }
-
- override def buildFilter(
- predicates: Seq[Expression],
- cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
- //essentially a noop
- (_: Int, b: Iterator[CachedBatch]) => b
- }
-}
-
-/**
- * Similar to ParquetFileFormat
- */
-private[rapids] class ParquetOutputFileFormat {
-
- def getRecordWriter(output: OutputFile, conf: Configuration): RecordWriter[Void, InternalRow] = {
- import ParquetOutputFormat._
-
- val blockSize = getLongBlockSize(conf)
- val maxPaddingSize =
- conf.getInt(MAX_PADDING_BYTES, ParquetWriter.MAX_PADDING_SIZE_DEFAULT)
- val validating = getValidation(conf)
-
- val writeSupport = new ParquetWriteSupport().asInstanceOf[WriteSupport[InternalRow]]
- val init = writeSupport.init(conf)
- val writer = new ParquetFileWriter(output, init.getSchema,
- Mode.CREATE, blockSize, maxPaddingSize)
- writer.start()
-
- val writerVersion =
- ParquetProperties.WriterVersion.fromString(conf.get(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString))
-
- val codecFactory = new CodecFactory(conf, getPageSize(conf))
-
- new ParquetRecordWriter[InternalRow](writer, writeSupport, init.getSchema,
- init.getExtraMetaData, blockSize, getPageSize(conf),
- codecFactory.getCompressor(CompressionCodecName.UNCOMPRESSED), getDictionaryPageSize(conf),
- getEnableDictionary(conf), validating, writerVersion,
- ParquetOutputFileFormat.getMemoryManager(conf))
-
- }
-}
-
-private object ParquetOutputFileFormat {
- var memoryManager: MemoryManager = _
- val DEFAULT_MEMORY_POOL_RATIO: Float = 0.95f
- val DEFAULT_MIN_MEMORY_ALLOCATION: Long = 1 * 1024 * 1024 // 1MB
-
- def getMemoryManager(conf: Configuration): MemoryManager = {
- synchronized {
- if (memoryManager == null) {
- import ParquetOutputFormat._
- val maxLoad = conf.getFloat(MEMORY_POOL_RATIO, DEFAULT_MEMORY_POOL_RATIO)
- val minAllocation = conf.getLong(MIN_MEMORY_ALLOCATION, DEFAULT_MIN_MEMORY_ALLOCATION)
- memoryManager = new MemoryManager(maxLoad, minAllocation)
- }
- }
- memoryManager
- }
-}
diff --git a/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/SparkBaseShims.scala b/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/SparkBaseShims.scala
index 186b96cb1e6..cc6941911b7 100644
--- a/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/SparkBaseShims.scala
+++ b/shims/spark312/src/main/scala/com/nvidia/spark/rapids/shims/spark312/SparkBaseShims.scala
@@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.shims.spark312
import java.net.URI
import java.nio.ByteBuffer
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.v2.Spark30XShims
import org.apache.arrow.memory.ReferenceManager
@@ -58,6 +59,7 @@ import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuB
import org.apache.spark.sql.rapids.execution.python.GpuPythonUDF
import org.apache.spark.sql.rapids.execution.python.shims.spark312._
import org.apache.spark.sql.rapids.shims.spark312._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
diff --git a/shims/spark312/src/main/scala/org/apache/spark/sql/rapids/shims/spark312/GpuInMemoryTableScanExec.scala b/shims/spark312/src/main/scala/org/apache/spark/sql/rapids/shims/spark312/GpuInMemoryTableScanExec.scala
deleted file mode 100644
index 80afc9bb634..00000000000
--- a/shims/spark312/src/main/scala/org/apache/spark/sql/rapids/shims/spark312/GpuInMemoryTableScanExec.scala
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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 org.apache.spark.sql.rapids.shims.spark312
-
-import com.nvidia.spark.rapids.{GpuExec, GpuMetric}
-import com.nvidia.spark.rapids.shims.spark312.ParquetCachedBatchSerializer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.physical.Partitioning
-import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
-import org.apache.spark.sql.vectorized.ColumnarBatch
-
-case class GpuInMemoryTableScanExec(
- attributes: Seq[Attribute],
- predicates: Seq[Expression],
- @transient relation: InMemoryRelation) extends LeafExecNode with GpuExec {
-
- override val nodeName: String = {
- relation.cacheBuilder.tableName match {
- case Some(_) =>
- "Scan " + relation.cacheBuilder.cachedName
- case _ =>
- super.nodeName
- }
- }
-
- override def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren
-
- override def doCanonicalize(): SparkPlan =
- copy(attributes = attributes.map(QueryPlan.normalizeExpressions(_, relation.output)),
- predicates = predicates.map(QueryPlan.normalizeExpressions(_, relation.output)),
- relation = relation.canonicalized.asInstanceOf[InMemoryRelation])
-
- override def vectorTypes: Option[Seq[String]] =
- relation.cacheBuilder.serializer.vectorTypes(attributes, conf)
-
- private lazy val columnarInputRDD: RDD[ColumnarBatch] = {
- val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS)
- val buffers = filteredCachedBatches()
- relation.cacheBuilder.serializer.asInstanceOf[ParquetCachedBatchSerializer]
- .gpuConvertCachedBatchToColumnarBatch(
- buffers,
- relation.output,
- attributes,
- conf).map { cb =>
- numOutputRows += cb.numRows()
- cb
- }
- }
-
- override def output: Seq[Attribute] = attributes
-
- private def updateAttribute(expr: Expression): Expression = {
- // attributes can be pruned so using relation's output.
- // E.g., relation.output is [id, item] but this scan's output can be [item] only.
- val attrMap = AttributeMap(relation.cachedPlan.output.zip(relation.output))
- expr.transform {
- case attr: Attribute => attrMap.getOrElse(attr, attr)
- }
- }
-
- // The cached version does not change the outputPartitioning of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputPartitioning: Partitioning = {
- relation.cachedPlan.outputPartitioning match {
- case e: Expression => updateAttribute(e).asInstanceOf[Partitioning]
- case other => other
- }
- }
-
- // The cached version does not change the outputOrdering of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputOrdering: Seq[SortOrder] =
- relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder])
-
- lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled
-
- // Accumulators used for testing purposes
- lazy val readPartitions = sparkContext.longAccumulator
- lazy val readBatches = sparkContext.longAccumulator
-
- private def filteredCachedBatches() = {
- // Right now just return the batch without filtering
- relation.cacheBuilder.cachedColumnBuffers
- }
-
- protected override def doExecute(): RDD[InternalRow] = {
- throw new UnsupportedOperationException("This Exec only deals with Columnar Data")
- }
-
- protected override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- columnarInputRDD
- }
-}
diff --git a/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/ParquetCachedBatchSerializer.scala b/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/ParquetCachedBatchSerializer.scala
deleted file mode 100644
index 33ec559767d..00000000000
--- a/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/ParquetCachedBatchSerializer.scala
+++ /dev/null
@@ -1,1521 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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.spark313
-
-import java.io.{InputStream, IOException}
-import java.lang.reflect.Method
-import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-
-import ai.rapids.cudf._
-import ai.rapids.cudf.ParquetWriterOptions.StatisticsFrequency
-import com.nvidia.spark.rapids._
-import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
-import com.nvidia.spark.rapids.RapidsPluginImplicits._
-import java.util
-import org.apache.commons.io.output.ByteArrayOutputStream
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.RecordWriter
-import org.apache.parquet.{HadoopReadOptions, ParquetReadOptions}
-import org.apache.parquet.column.{ColumnDescriptor, ParquetProperties}
-import org.apache.parquet.hadoop.{CodecFactory, MemoryManager, ParquetFileReader, ParquetFileWriter, ParquetInputFormat, ParquetOutputFormat, ParquetRecordWriter, ParquetWriter}
-import org.apache.parquet.hadoop.ParquetFileWriter.Mode
-import org.apache.parquet.hadoop.api.WriteSupport
-import org.apache.parquet.hadoop.metadata.CompressionCodecName
-import org.apache.parquet.io.{DelegatingPositionOutputStream, DelegatingSeekableInputStream, InputFile, OutputFile, PositionOutputStream, SeekableInputStream}
-import org.apache.parquet.schema.{MessageType, Type}
-
-import org.apache.spark.TaskContext
-import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{vectorized, SparkSession}
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericInternalRow, SpecializedGetters, UnsafeRow}
-import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
-import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
-import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer}
-import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetToSparkSchemaConverter, ParquetWriteSupport, SparkToParquetSchemaConverter, VectorizedColumnReader}
-import org.apache.spark.sql.execution.datasources.parquet.rapids.shims.spark313.ParquetRecordMaterializer
-import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.unsafe.types.CalendarInterval
-
-/**
- * copied from Spark org.apache.spark.util.ByteBufferInputStream
- */
-private class ByteBufferInputStream(private var buffer: ByteBuffer)
- extends InputStream {
-
- override def read(): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- buffer.get() & 0xFF
- }
- }
-
- override def read(dest: Array[Byte]): Int = {
- read(dest, 0, dest.length)
- }
-
- override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
- if (buffer == null || buffer.remaining() == 0) {
- cleanUp()
- -1
- } else {
- val amountToGet = math.min(buffer.remaining(), length)
- buffer.get(dest, offset, amountToGet)
- amountToGet
- }
- }
-
- override def skip(bytes: Long): Long = {
- if (buffer != null) {
- val amountToSkip = math.min(bytes, buffer.remaining).toInt
- buffer.position(buffer.position() + amountToSkip)
- if (buffer.remaining() == 0) {
- cleanUp()
- }
- amountToSkip
- } else {
- 0L
- }
- }
-
- /**
- * Clean up the buffer, and potentially dispose of it using StorageUtils.dispose().
- */
- private def cleanUp(): Unit = {
- if (buffer != null) {
- buffer = null
- }
- }
-}
-
-class ByteArrayInputFile(buff: Array[Byte]) extends InputFile {
-
- override def getLength: Long = buff.length
-
- override def newStream(): SeekableInputStream = {
- val byteBuffer = ByteBuffer.wrap(buff)
- new DelegatingSeekableInputStream(new ByteBufferInputStream(byteBuffer)) {
- override def getPos: Long = byteBuffer.position()
-
- override def seek(newPos: Long): Unit = {
- if (newPos > Int.MaxValue || newPos < Int.MinValue) {
- throw new IllegalStateException("seek value is out of supported range " + newPos)
- }
- byteBuffer.position(newPos.toInt)
- }
- }
- }
-}
-
-private object ByteArrayOutputFile {
- val BLOCK_SIZE: Int = 32 * 1024 * 1024 // 32M
-}
-
-private class ByteArrayOutputFile(stream: ByteArrayOutputStream) extends OutputFile {
- override def create(blockSizeHint: Long): PositionOutputStream = {
- new DelegatingPositionOutputStream(stream) {
- var pos = 0
-
- override def getPos: Long = pos
-
- override def write(b: Int): Unit = {
- super.write(b)
- pos += Integer.BYTES
- }
-
- override def write(b: Array[Byte]): Unit = {
- super.write(b)
- pos += b.length
- }
-
- override def write(b: Array[Byte], off: Int, len: Int): Unit = {
- super.write(b, off, len)
- pos += len
- }
- }
- }
-
- override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream =
- throw new UnsupportedOperationException("Don't need to overwrite")
-
- override def supportsBlockSize(): Boolean = true
-
- override def defaultBlockSize(): Long = ByteArrayOutputFile.BLOCK_SIZE
-}
-
-private class ParquetBufferConsumer(val numRows: Int) extends HostBufferConsumer with
- AutoCloseable {
- @transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
- private var buffer: Array[Byte] = _
-
- override def handleBuffer(buffer: HostMemoryBuffer, len: Long): Unit = {
- offHeapBuffers += Tuple2(buffer, len)
- }
-
- def getBuffer: Array[Byte] = {
- if (buffer == null) {
- writeBuffers()
- }
- buffer
- }
-
- def close(): Unit = {
- if (buffer == null) {
- writeBuffers()
- }
- }
-
- private def writeBuffers(): Unit = {
- val toProcess = offHeapBuffers.dequeueAll(_ => true)
- // We are making sure the input is smaller than 2gb so the parquet written should never be more
- // than Int.MAX_SIZE.
- val bytes = toProcess.map(_._2).sum
-
- // for now assert bytes are less than Int.MaxValue
- assert(bytes <= Int.MaxValue)
- buffer = new Array(bytes.toInt)
- try {
- var offset: Int = 0
- toProcess.foreach(ops => {
- val origBuffer = ops._1
- val len = ops._2.toInt
- origBuffer.asByteBuffer().get(buffer, offset, len)
- offset = offset + len
- })
- } finally {
- toProcess.map(_._1).safeClose()
- }
- }
-}
-
-private object ParquetCachedBatch {
- def apply(parquetBuff: ParquetBufferConsumer): ParquetCachedBatch = {
- new ParquetCachedBatch(parquetBuff.numRows, parquetBuff.getBuffer)
- }
-}
-
-case class ParquetCachedBatch(
- numRows: Int,
- buffer: Array[Byte]) extends CachedBatch {
- override def sizeInBytes: Long = buffer.length
-}
-
-/**
- * Spark wants the producer to close the batch. We have a listener in this iterator that will close
- * the batch after the task is completed
- */
-private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
- Iterator[ColumnarBatch] {
- var cb: ColumnarBatch = _
-
- private def closeCurrentBatch(): Unit = {
- if (cb != null) {
- cb.close()
- cb = null
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: 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 = schema.fields.forall { f =>
- // only check spark b/c if we are on the GPU then we will be calling the gpu method regardless
- isTypeSupportedByColumnarSparkParquetWriter(f.dataType) || f.dataType == DataTypes.NullType
- }
-
- private def isTypeSupportedByColumnarSparkParquetWriter(dataType: DataType): Boolean = {
- // Columnar writer in Spark only supports AtomicTypes ATM
- dataType match {
- case TimestampType | StringType | BooleanType | DateType | BinaryType |
- DoubleType | FloatType | ByteType | IntegerType | LongType | ShortType => true
- case _: DecimalType => true
- case _ => false
- }
- }
-
- def isSchemaSupportedByCudf(schema: Seq[Attribute]): Boolean = {
- schema.forall(field => isSupportedByCudf(field.dataType))
- }
-
- def isSupportedByCudf(dataType: DataType): Boolean = {
- dataType match {
- // TODO: when arrays are supported for cudf writes add it here.
- // https://github.com/NVIDIA/spark-rapids/issues/2054
- case s: StructType => s.forall(field => isSupportedByCudf(field.dataType))
- case _ => GpuColumnVector.isNonNestedSupportedType(dataType)
- }
- }
-
- /**
- * This method checks if the datatype passed is officially supported by parquet.
- *
- * Please refer to https://github.com/apache/parquet-format/blob/master/LogicalTypes.md to see
- * the what types are supported by parquet
- */
- def isTypeSupportedByParquet(dataType: DataType): Boolean = {
- dataType match {
- case CalendarIntervalType | NullType => false
- case s: StructType => s.forall(field => isTypeSupportedByParquet(field.dataType))
- case ArrayType(elementType, _) => isTypeSupportedByParquet(elementType)
- case MapType(keyType, valueType, _) => isTypeSupportedByParquet(keyType) &&
- isTypeSupportedByParquet(valueType)
- case d: DecimalType if d.scale < 0 => false
- case _ => 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- val structSchema = schemaWithUnambiguousNames.toStructType
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- def putOnGpuIfNeeded(batch: ColumnarBatch): ColumnarBatch = {
- if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
- val s: StructType = structSchema
- val gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows()).build(batch.numRows())
- batch.close()
- gpuCB
- } else {
- batch
- }
- }
-
- input.flatMap(batch => {
- if (batch.numCols() == 0) {
- List(ParquetCachedBatch(batch.numRows(), new Array[Byte](0)))
- } else {
- withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
- compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
- bytesAllowedPerBatch)
- }
- }
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[ColumnarBatch](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getColumnarBatchToCachedBatchIterator
- }
- }
- }
-
- private[rapids] def compressColumnarBatchWithParquet(
- oldGpuCB: ColumnarBatch,
- schema: StructType,
- origSchema: StructType,
- bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
- val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
- oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
- .getBase.getDeviceMemorySize / oldGpuCB.numRows()
- }.sum
-
- val columns = for (i <- 0 until oldGpuCB.numCols()) yield {
- val gpuVector = oldGpuCB.column(i).asInstanceOf[GpuColumnVector]
- var dataType = origSchema(i).dataType
- val v = ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(gpuVector.getBase,
- origSchema(i).dataType,
- // we are checking for scale > 0 because cudf and spark refer to scales as opposites
- // e.g. scale = -3 in Spark is scale = 3 in cudf
- (_, cv) => cv.getType.isDecimalType && cv.getType.getScale > 0,
- (_, cv) => {
- if (cv.getType.isBackedByLong) {
- dataType = LongType
- cv.bitCastTo(DType.INT64)
- } else {
- dataType = IntegerType
- cv.bitCastTo(DType.INT32)
- }
- }
- )
- GpuColumnVector.from(v, schema(i).dataType)
- }
- withResource(new ColumnarBatch(columns.toArray, oldGpuCB.numRows())) { gpuCB =>
- val rowsAllowedInBatch = (bytesAllowedPerBatch / estimatedRowSize).toInt
- val splitIndices = scala.Range(rowsAllowedInBatch, gpuCB.numRows(), rowsAllowedInBatch)
- val buffers = new ListBuffer[ParquetCachedBatch]
- if (splitIndices.nonEmpty) {
- val splitVectors = new ListBuffer[Array[ColumnVector]]
- try {
- for (index <- 0 until gpuCB.numCols()) {
- splitVectors +=
- gpuCB.column(index).asInstanceOf[GpuColumnVector].getBase.split(splitIndices: _*)
- }
-
- // Splitting the table
- // e.g. T0 = {col1, col2,...,coln} => split columns into 'm' cols =>
- // T00= {splitCol1(0), splitCol2(0),...,splitColn(0)}
- // T01= {splitCol1(1), splitCol2(1),...,splitColn(1)}
- // ...
- // T0m= {splitCol1(m), splitCol2(m),...,splitColn(m)}
- def makeTableForIndex(i: Int): Table = {
- val columns = splitVectors.indices.map(j => splitVectors(j)(i))
- new Table(columns: _*)
- }
-
- for (i <- splitVectors.head.indices) {
- withResource(makeTableForIndex(i)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- } finally {
- splitVectors.foreach(array => array.safeClose())
- }
- } else {
- withResource(GpuColumnVector.from(gpuCB)) { table =>
- val buffer = writeTableToCachedBatch(table, schema)
- buffers += ParquetCachedBatch(buffer)
- }
- }
- buffers.toList
- }
- }
-
- private def writeTableToCachedBatch(
- table: Table,
- schema: StructType): ParquetBufferConsumer = {
- val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
- val opts = GpuParquetFileFormat
- .parquetWriterOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
- .withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
- withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
- writer.write(table)
- }
- buffer
- }
-
- /**
- * This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying 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.
- */
- def gpuConvertCachedBatchToColumnarBatch(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- conf: SQLConf): RDD[ColumnarBatch] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- convertCachedBatchToColumnarInternal(
- input,
- cachedSchemaWithNames,
- selectedSchemaWithNames,
- newSelectedAttributes)
- }
-
- private def convertCachedBatchToColumnarInternal(
- input: RDD[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- originalSelectedAttributes: Seq[Attribute]): RDD[ColumnarBatch] = {
-
- val cbRdd: RDD[ColumnarBatch] = input.map {
- case parquetCB: ParquetCachedBatch =>
- val parquetOptions = ParquetOptions.builder()
- .includeColumn(selectedAttributes.map(_.name).asJavaCollection).build()
- withResource(Table.readParquet(parquetOptions, parquetCB.buffer, 0,
- parquetCB.sizeInBytes)) { table =>
- withResource {
- for (i <- 0 until table.getNumberOfColumns) yield {
- ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(table.getColumn(i),
- originalSelectedAttributes(i).dataType,
- (dataType, _) => dataType match {
- case d: DecimalType if d.scale < 0 => true
- case _ => false
- },
- (dataType, cv) => {
- //TODO: why do we have to copy to a vector
- dataType match {
- case d: DecimalType =>
- withResource(cv.bitCastTo(DecimalUtil.createCudfDecimal(d))) {
- _.copyToColumnVector()
- }
- case _ =>
- throw new IllegalStateException("We don't cast any type besides Decimal " +
- "with scale < 0")
- }
- }
- )
- }
- } { col =>
- withResource(new Table(col: _*)) { t =>
- GpuColumnVector.from(t, originalSelectedAttributes.map(_.dataType).toArray)
- }
- }
- }
- case _ =>
- throw new IllegalStateException("I don't know how to convert this batch")
- }
- cbRdd
- }
-
- private def getSelectedSchemaFromCachedSchema(
- selectedAttributes: Seq[Attribute],
- cacheAttributes: Seq[Attribute]): Seq[Attribute] = {
- selectedAttributes.map {
- a => cacheAttributes(cacheAttributes.map(_.exprId).indexOf(a.exprId))
- }
- }
-
- /**
- * 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] = {
- // optimize
- val newSelectedAttributes = if (selectedAttributes.isEmpty) {
- cacheAttributes
- } else {
- selectedAttributes
- }
- val rapidsConf = new RapidsConf(conf)
- val (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
- if (rapidsConf.isSqlEnabled &&
- isSchemaSupportedByCudf(cachedSchemaWithNames)) {
- val batches = convertCachedBatchToColumnarInternal(input, cachedSchemaWithNames,
- selectedSchemaWithNames, newSelectedAttributes)
- val cbRdd = batches.map(batch => {
- withResource(batch) { gpuBatch =>
- val cols = GpuColumnVector.extractColumns(gpuBatch)
- new ColumnarBatch(cols.safeMap(_.copyToHost()).toArray, gpuBatch.numRows())
- }
- })
- cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter))
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, newSelectedAttributes, broadcastedConf).getColumnBatchIterator
- }
- }
- }
- }
-
- /**
- * 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 (cachedSchemaWithNames, selectedSchemaWithNames) =
- getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes)
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter => {
- new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
- cacheAttributes, selectedAttributes, broadcastedConf).getInternalRowIterator
- }
- }
- }
-
- private abstract class UnsupportedDataHandlerIterator extends Iterator[InternalRow] {
-
- def handleInternalRow(schema: Seq[Attribute], row: InternalRow, newRow: InternalRow): Unit
-
- def handleInterval(data: SpecializedGetters, index: Int): Any
-
- def handleStruct(
- data: InternalRow,
- origSchema: StructType,
- supportedSchema: StructType): InternalRow = {
- val structRow = InternalRow.fromSeq(supportedSchema)
- handleInternalRow(origSchema.map(field =>
- AttributeReference(field.name, field.dataType, field.nullable)()), data, structRow)
- structRow
- }
-
- def handleMap(
- keyType: DataType,
- valueType: DataType,
- mapData: MapData): MapData = {
- val keyData = mapData.keyArray()
- val newKeyData = handleArray(keyType, keyData)
- val valueData = mapData.valueArray()
- val newValueData = handleArray(valueType, valueData)
- new ArrayBasedMapData(newKeyData, newValueData)
- }
-
- def handleArray(
- dataType: DataType,
- arrayData: ArrayData): ArrayData = {
- dataType match {
- case s@StructType(_) =>
- val listBuffer = new ListBuffer[InternalRow]()
- val supportedSchema = mapping(dataType).asInstanceOf[StructType]
- arrayData.foreach(supportedSchema, (_, data) => {
- val structRow =
- handleStruct(data.asInstanceOf[InternalRow], s, s)
- listBuffer += structRow.copy()
- })
- new GenericArrayData(listBuffer)
-
- case ArrayType(elementType, _) =>
- val arrayList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val subArrayData = arrayData.getArray(i)
- arrayList.append(handleArray(elementType, subArrayData))
- }
- new GenericArrayData(arrayList)
-
- case m@MapType(_, _, _) =>
- val mapList =
- new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val mapData = arrayData.getMap(i)
- mapList.append(handleMap(m.keyType, m.valueType, mapData))
- }
- new GenericArrayData(mapList)
-
- case CalendarIntervalType =>
- val citList = new ListBuffer[Any]()
- scala.Range(0, arrayData.numElements()).foreach { i =>
- val citRow = handleInterval(arrayData, i)
- citList += citRow
- }
- new GenericArrayData(citList)
-
- case _ =>
- arrayData
- }
- }
- }
-
- /**
- * Consumes the Iterator[CachedBatch] to return either Iterator[ColumnarBatch] or
- * Iterator[InternalRow]
- */
- private class CachedBatchIteratorConsumer(
- cbIter: Iterator[CachedBatch],
- cacheAttributes: Seq[Attribute],
- selectedAttributes: Seq[Attribute],
- origCacheSchema: Seq[Attribute],
- origRequestedSchema: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val hadoopConf: Configuration = getHadoopConf(origRequestedSchema.toStructType, conf)
- val options: ParquetReadOptions = HadoopReadOptions.builder(hadoopConf).build()
- /**
- * We are getting this method using reflection because its a package-private
- */
- val readBatchMethod: Method =
- classOf[VectorizedColumnReader].getDeclaredMethod("readBatch", Integer.TYPE,
- classOf[WritableColumnVector])
- readBatchMethod.setAccessible(true)
-
- def getInternalRowIterator: Iterator[InternalRow] = {
-
- /**
- * This iterator converts an iterator[CachedBatch] to an iterator[InternalRow].
- *
- * This makes it unlike a regular iterator because CachedBatch => InternalRow* is a 1-n
- * relation. The way we have implemented this is to first go through the
- * iterator[CachedBatch] (cbIter) to look for a valid iterator (iter) i.e. hasNext() => true.
- * Then every time next() is called we return a single InternalRow from iter. When
- * iter.hasNext() => false, we find the next valid iterator in cbIter and the process
- * continues as above.
- */
- new Iterator[InternalRow]() {
-
- var iter: Iterator[InternalRow] = _
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = convertCachedBatchToInternalRowIter
- }
- iter != null && iter.hasNext
- }
-
- override def next(): InternalRow = {
- // will return the next InternalRow if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- /**
- * This method converts a CachedBatch to an iterator of InternalRows.
- */
- private def convertCachedBatchToInternalRowIter: Iterator[InternalRow] = {
- val parquetCachedBatch = cbIter.next().asInstanceOf[ParquetCachedBatch]
- val inputFile = new ByteArrayInputFile(parquetCachedBatch.buffer)
- withResource(ParquetFileReader.open(inputFile, options)) { parquetFileReader =>
- val parquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val hasUnsupportedType = origCacheSchema.exists { field =>
- !isTypeSupportedByParquet(field.dataType)
- }
-
- val unsafeRows = new ArrayBuffer[InternalRow]
- import org.apache.parquet.io.ColumnIOFactory
- var pages = parquetFileReader.readNextRowGroup()
- while (pages != null) {
- val rows = pages.getRowCount
- val columnIO = new ColumnIOFactory().getColumnIO(parquetSchema)
- val recordReader =
- columnIO.getRecordReader(pages, new ParquetRecordMaterializer(parquetSchema,
- cacheAttributes.toStructType,
- new ParquetToSparkSchemaConverter(hadoopConf), None /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED))
- for (_ <- 0 until rows.toInt) {
- val row = recordReader.read
- unsafeRows += row.copy()
- }
- pages = parquetFileReader.readNextRowGroup()
- }
-
- val iter = unsafeRows.iterator
- val unsafeProjection =
- GenerateUnsafeProjection.generate(selectedAttributes, cacheAttributes)
- if (hasUnsupportedType) {
- new UnsupportedDataHandlerIterator() {
- val wrappedIter: Iterator[InternalRow] = iter
- val newRow = new GenericInternalRow(cacheAttributes.length)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- //read a row and convert it to what the caller is expecting
- val row = wrappedIter.next()
- handleInternalRow(origCacheSchema, row, newRow)
- val unsafeProjection =
- GenerateUnsafeProjection.generate(origRequestedSchema, origCacheSchema)
- unsafeProjection.apply(newRow)
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): CalendarInterval = {
- if (data.isNullAt(index)) {
- null
- } else {
- val structData = data.getStruct(index, 3)
- new CalendarInterval(structData.getInt(0),
- structData.getInt(1), structData.getLong(2))
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val supportedSchema = mapping(dataType)
- .asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, supportedSchema.size), s, s)
- newRow.update(index, structRow)
-
- case a@ArrayType(_, _) =>
- val arrayData = row.getArray(index)
- newRow.update(index, handleArray(a.elementType, arrayData))
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- newRow.update(index, handleMap(keyType, valueType, mapData))
-
- case CalendarIntervalType =>
- val interval = handleInterval(row, index)
- if (interval == null) {
- newRow.setNullAt(index)
- } else {
- newRow.setInterval(index, interval)
- }
- case d: DecimalType =>
- if (row.isNullAt(index)) {
- newRow.setDecimal(index, null, d.precision)
- } else {
- val dec = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- Decimal(row.getInt(index).toLong, d.precision, d.scale)
- } else {
- Decimal(row.getLong(index), d.precision, d.scale)
- }
- newRow.update(index, dec)
- }
- case NullType =>
- newRow.setNullAt(index)
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- } else {
- iter.map(unsafeProjection)
- }
- }
- }
- }
- }
-
- private class CurrentBatchIterator(val parquetCachedBatch: ParquetCachedBatch)
- extends Iterator[ColumnarBatch] with AutoCloseable {
-
- val capacity = conf.parquetVectorizedReaderBatchSize
- var columnReaders: Array[VectorizedColumnReader] = _
- val columnVectors: Array[OffHeapColumnVector] =
- OffHeapColumnVector.allocateColumns(capacity, selectedAttributes.toStructType)
- val columnarBatch = new ColumnarBatch(columnVectors
- .asInstanceOf[Array[vectorized.ColumnVector]])
- var rowsReturned: Long = 0L
- var numBatched = 0
- var batchIdx = 0
- var totalCountLoadedSoFar: Long = 0
- val parquetFileReader =
- ParquetFileReader.open(new ByteArrayInputFile(parquetCachedBatch.buffer), options)
- val (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache) = {
- val parquetToSparkSchemaConverter = new ParquetToSparkSchemaConverter(hadoopConf)
- // we are getting parquet schema and then converting it to catalyst schema
- // because catalyst schema that we get from Spark doesn't have the exact schema expected
- // by the columnar parquet reader
- val inMemCacheParquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
- val inMemCacheSparkSchema = parquetToSparkSchemaConverter.convert(inMemCacheParquetSchema)
-
- val totalRowCount = parquetFileReader.getRowGroups.asScala.map(_.getRowCount).sum
- val sparkToParquetSchemaConverter = new SparkToParquetSchemaConverter(hadoopConf)
- val inMemReqSparkSchema = StructType(selectedAttributes.toStructType.map { field =>
- inMemCacheSparkSchema.fields(inMemCacheSparkSchema.fieldIndex(field.name))
- })
- val inMemReqParquetSchema = sparkToParquetSchemaConverter.convert(inMemReqSparkSchema)
- val columnsRequested: util.List[ColumnDescriptor] = inMemReqParquetSchema.getColumns
-
- val reqSparkSchemaInCacheOrder = StructType(inMemCacheSparkSchema.filter(f =>
- inMemReqSparkSchema.fields.exists(f0 => f0.name.equals(f.name))))
-
- // There could be a case especially in a distributed environment where the requestedSchema
- // and cacheSchema are not in the same order. We need to create a map so we can guarantee
- // that we writing to the correct columnVector
- val cacheSchemaToReqSchemaMap: Map[Int, Int] =
- reqSparkSchemaInCacheOrder.indices.map { index =>
- index -> inMemReqSparkSchema.fields.indexOf(reqSparkSchemaInCacheOrder.fields(index))
- }.toMap
-
- val reqParquetSchemaInCacheOrder =
- sparkToParquetSchemaConverter.convert(reqSparkSchemaInCacheOrder)
-
- // reset spark schema calculated from parquet schema
- hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, inMemReqSparkSchema.json)
- hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, inMemReqSparkSchema.json)
-
- val columnsInCache: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
- val typesInCache: util.List[Type] = reqParquetSchemaInCacheOrder.asGroupType.getFields
- val missingColumns = new Array[Boolean](inMemReqParquetSchema.getFieldCount)
-
- // initialize missingColumns to cover the case where requested column isn't present in the
- // cache, which should never happen but just in case it does
- val paths: util.List[Array[String]] = inMemReqParquetSchema.getPaths
-
- for (i <- 0 until inMemReqParquetSchema.getFieldCount) {
- val t = inMemReqParquetSchema.getFields.get(i)
- if (!t.isPrimitive || t.isRepetition(Type.Repetition.REPEATED)) {
- throw new UnsupportedOperationException("Complex types not supported.")
- }
- val colPath = paths.get(i)
- if (inMemCacheParquetSchema.containsPath(colPath)) {
- val fd = inMemCacheParquetSchema.getColumnDescription(colPath)
- if (!(fd == columnsRequested.get(i))) {
- throw new UnsupportedOperationException("Schema evolution not supported.")
- }
- missingColumns(i) = false
- } else {
- if (columnsRequested.get(i).getMaxDefinitionLevel == 0) {
- // Column is missing in data but the required data is non-nullable.
- // This file is invalid.
- throw new IOException(s"Required column is missing in data file: ${colPath.toList}")
- }
- missingColumns(i) = true
- }
- }
-
- for (i <- missingColumns.indices) {
- if (missingColumns(i)) {
- columnVectors(i).putNulls(0, capacity)
- columnVectors(i).setIsConstant()
- }
- }
-
- (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
- columnsInCache, typesInCache)
- }
-
- @throws[IOException]
- def checkEndOfRowGroup(): Unit = {
- if (rowsReturned != totalCountLoadedSoFar) return
- val pages = parquetFileReader.readNextRowGroup
- if (pages == null) {
- throw new IOException("expecting more rows but reached last" +
- " block. Read " + rowsReturned + " out of " + totalRowCount)
- }
- columnReaders = new Array[VectorizedColumnReader](columnsRequested.size)
- for (i <- 0 until columnsRequested.size) {
- if (!missingColumns(i)) {
- columnReaders(i) =
- new VectorizedColumnReader(
- columnsInCache.get(i),
- typesInCache.get(i).getOriginalType,
- pages.getPageReader(columnsInCache.get(i)),
- null /*convertTz*/ ,
- LegacyBehaviorPolicy.CORRECTED.toString,
- LegacyBehaviorPolicy.EXCEPTION.toString)
- }
- }
- totalCountLoadedSoFar += pages.getRowCount
- }
-
- /**
- * Read the next RowGroup and read each column and return the columnarBatch
- */
- def nextBatch: Boolean = {
- for (vector <- columnVectors) {
- vector.reset()
- }
- columnarBatch.setNumRows(0)
- if (rowsReturned >= totalRowCount) return false
- checkEndOfRowGroup()
- val num = Math.min(capacity.toLong, totalCountLoadedSoFar - rowsReturned).toInt
- for (i <- columnReaders.indices) {
- if (columnReaders(i) != null) {
- readBatchMethod.invoke(columnReaders(i), num.asInstanceOf[AnyRef],
- columnVectors(cacheSchemaToReqSchemaMap(i)).asInstanceOf[AnyRef])
- }
- }
- rowsReturned += num
- columnarBatch.setNumRows(num)
- numBatched = num
- batchIdx = 0
- true
- }
-
- override def hasNext: Boolean = rowsReturned < totalRowCount
-
- override def next(): ColumnarBatch = {
- if (nextBatch) {
- // FYI, A very IMPORTANT thing to note is that we are returning the columnar batch
- // as-is i.e. this batch has NullTypes saved as IntegerTypes with null values. The
- // way Spark optimizes the read of NullTypes makes this work without having to rip out
- // the IntegerType column to be replaced by a NullType column. This could change in
- // future and will affect this code.
- columnarBatch
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
-
- TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => {
- close()
- })
-
- override def close(): Unit = {
- parquetFileReader.close()
- }
- }
-
- /**
- * This method returns a ColumnarBatch iterator over a CachedBatch.
- * Each CachedBatch => ColumnarBatch is a 1-1 conversion so its pretty straight forward
- */
- def getColumnBatchIterator: Iterator[ColumnarBatch] = new Iterator[ColumnarBatch] {
- var iter = getIterator
-
- def getIterator: Iterator[ColumnarBatch] = {
- if (!cbIter.hasNext) {
- Iterator.empty
- } else {
- new CurrentBatchIterator(cbIter.next().asInstanceOf[ParquetCachedBatch])
- }
- }
-
- override def hasNext: Boolean = {
- // go over the batch and get the next non-degenerate iterator
- // and return if it hasNext
- while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
- iter = getIterator
- }
- iter != null && iter.hasNext
- }
-
- override def next(): ColumnarBatch = {
- // will return the next ColumnarBatch if hasNext() is true, otherwise throw
- if (hasNext) {
- iter.next()
- } else {
- throw new NoSuchElementException("no elements found")
- }
- }
- }
- }
-
- private def getConfFromMap(sharedConf: Broadcast[Map[String, String]]): SQLConf = {
- val conf = new SQLConf()
- sharedConf.value.foreach { case (k, v) => conf.setConfString(k, v) }
- conf
- }
-
- private val intervalStructType = new StructType()
- .add("_days", IntegerType)
- .add("_months", IntegerType)
- .add("_ms", LongType)
-
- def getBytesAllowedPerBatch(conf: SQLConf): Long = {
- val gpuBatchSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf)
- // we are rough estimating 0.5% as meta_data_size. we can do better estimation in future
- val approxMetaDataSizeBytes = gpuBatchSize * 0.5/100
- (gpuBatchSize - approxMetaDataSizeBytes).toLong
- }
-
- /**
- * This is a private helper class to return Iterator to convert InternalRow or ColumnarBatch to
- * CachedBatch. There is no type checking so if the type of T is anything besides InternalRow
- * or ColumnarBatch then the behavior is undefined.
- *
- * @param iter - an iterator over InternalRow or ColumnarBatch
- * @param cachedAttributes - Schema of the cached batch
- * @param sharedConf - SQL conf
- * @tparam T - Strictly either InternalRow or ColumnarBatch
- */
- private[rapids] class CachedBatchIteratorProducer[T](
- iter: Iterator[T],
- cachedAttributes: Seq[Attribute],
- origCachedAttributes: Seq[Attribute],
- sharedConf: Broadcast[Map[String, String]]) {
-
- val conf: SQLConf = getConfFromMap(sharedConf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val hadoopConf: Configuration = getHadoopConf(cachedAttributes.toStructType, conf)
-
- def getInternalRowToCachedBatchIterator: Iterator[CachedBatch] = {
- new InternalRowToCachedBatchIterator
- }
-
- def getColumnarBatchToCachedBatchIterator: Iterator[CachedBatch] = {
- new ColumnarBatchToCachedBatchIterator
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[InternalRow]. This is a n-1
- * relationship. Each partition represents a single parquet file, so we encode it
- * and return the CachedBatch when next is called.
- */
- class InternalRowToCachedBatchIterator extends Iterator[CachedBatch]() {
-
- var parquetOutputFileFormat = new ParquetOutputFileFormat()
-
- // For testing only
- private[rapids] def setParquetOutputFileFormat(p: ParquetOutputFileFormat): Unit = {
- parquetOutputFileFormat = p
- }
-
- // is there a type that spark doesn't support by default in the schema?
- val hasUnsupportedType: Boolean = origCachedAttributes.exists { attribute =>
- !isTypeSupportedByParquet(attribute.dataType)
- }
-
- def getIterator: Iterator[InternalRow] = {
- if (!hasUnsupportedType) {
- iter.asInstanceOf[Iterator[InternalRow]]
- } else {
- new UnsupportedDataHandlerIterator {
-
- val wrappedIter: Iterator[InternalRow] = iter.asInstanceOf[Iterator[InternalRow]]
-
- val newRow: InternalRow = InternalRow.fromSeq(cachedAttributes)
-
- override def hasNext: Boolean = wrappedIter.hasNext
-
- override def next(): InternalRow = {
- val row = wrappedIter.next()
- handleInternalRow(origCachedAttributes, row, newRow)
- newRow
- }
-
- override def handleInterval(
- data: SpecializedGetters,
- index: Int): InternalRow = {
- val citRow = InternalRow(IntegerType, IntegerType, LongType)
- if (data.isNullAt(index)) {
- null
- } else {
- val cit = data.getInterval(index)
- citRow.setInt(0, cit.months)
- citRow.setInt(1, cit.days)
- citRow.setLong(2, cit.microseconds)
- citRow
- }
- }
-
- override def handleInternalRow(
- schema: Seq[Attribute],
- row: InternalRow,
- newRow: InternalRow): Unit = {
- schema.indices.foreach { index =>
- val dataType = schema(index).dataType
- if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
- dataType == NullType ||
- (dataType.isInstanceOf[DecimalType]
- && dataType.asInstanceOf[DecimalType].scale < 0)) {
- if (row.isNullAt(index)) {
- newRow.setNullAt(index)
- } else {
- dataType match {
- case s@StructType(_) =>
- val newSchema = mapping(dataType).asInstanceOf[StructType]
- val structRow =
- handleStruct(row.getStruct(index, s.fields.length), s, newSchema)
- newRow.update(index, structRow)
-
- case ArrayType(arrayDataType, _) =>
- val arrayData = row.getArray(index)
- val newArrayData = handleArray(arrayDataType, arrayData)
- newRow.update(index, newArrayData)
-
- case MapType(keyType, valueType, _) =>
- val mapData = row.getMap(index)
- val map = handleMap(keyType, valueType, mapData)
- newRow.update(index, map)
-
- case CalendarIntervalType =>
- val structData: InternalRow = handleInterval(row, index)
- if (structData == null) {
- newRow.setNullAt(index)
- } else {
- newRow.update(index, structData)
- }
-
- case d: DecimalType if d.scale < 0 =>
- if (d.precision <= Decimal.MAX_INT_DIGITS) {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong.toInt)
- } else {
- newRow.update(index, row.getDecimal(index, d.precision, d.scale)
- .toUnscaledLong)
- }
-
- case _ =>
- newRow.update(index, row.get(index, dataType))
- }
- }
- } else {
- newRow.update(index, row.get(index, dataType))
- }
- }
- }
- }
- }
- }
-
- override def hasNext: Boolean = queue.nonEmpty || iter.hasNext
-
- private val queue = new mutable.Queue[CachedBatch]()
-
- //estimate the size of a row
- val estimatedSize: Int = cachedAttributes.map { attr =>
- attr.dataType.defaultSize
- }.sum
-
- override def next(): CachedBatch = {
- if (queue.isEmpty) {
- // to store a row if we have read it but there is no room in the parquet file to put it
- // we will put it in the next CachedBatch
- var leftOverRow: Option[InternalRow] = None
- val rowIterator = getIterator
- while (rowIterator.hasNext || leftOverRow.nonEmpty) {
- // Each partition will be a single parquet file
- var rows = 0
- // at least a single block
- val stream = new ByteArrayOutputStream(ByteArrayOutputFile.BLOCK_SIZE)
- val outputFile: OutputFile = new ByteArrayOutputFile(stream)
- conf.setConfString(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
- val recordWriter = SQLConf.withExistingConf(conf) {
- parquetOutputFileFormat.getRecordWriter(outputFile, hadoopConf)
- }
- var totalSize = 0
- while ((rowIterator.hasNext || leftOverRow.nonEmpty)
- && totalSize < bytesAllowedPerBatch) {
-
- val row = if (leftOverRow.nonEmpty) {
- val a = leftOverRow.get
- leftOverRow = None // reset value
- a
- } else {
- rowIterator.next()
- }
- totalSize += {
- row match {
- case r: UnsafeRow =>
- r.getSizeInBytes
- case _ =>
- estimatedSize
- }
- }
- if (totalSize <= bytesAllowedPerBatch) {
- rows += 1
- if (rows < 0) {
- throw new IllegalStateException("CachedBatch doesn't support rows larger " +
- "than Int.MaxValue")
- }
- recordWriter.write(null, row)
- } else {
- leftOverRow = Some(if (row.isInstanceOf[UnsafeRow]) {
- row.copy()
- } else {
- row
- })
- }
- }
- // passing null as context isn't used in this method
- recordWriter.close(null)
- queue += ParquetCachedBatch(rows, stream.toByteArray)
- }
- }
- queue.dequeue()
- }
- }
-
- /**
- * This class produces an Iterator[CachedBatch] from Iterator[ColumnarBatch]. This is a 1-1
- * relationship. Each ColumnarBatch is converted to a single ParquetCachedBatch when next()
- * is called on this iterator
- */
- class ColumnarBatchToCachedBatchIterator extends InternalRowToCachedBatchIterator {
- override def getIterator: Iterator[InternalRow] = {
-
- new Iterator[InternalRow] {
- // We have to check for null context because of the unit test
- Option(TaskContext.get).foreach(_.addTaskCompletionListener[Unit](_ => hostBatch.close()))
-
- val batch: ColumnarBatch = iter.asInstanceOf[Iterator[ColumnarBatch]].next
- val hostBatch = if (batch.column(0).isInstanceOf[GpuColumnVector]) {
- withResource(batch) { batch =>
- new ColumnarBatch(batch.safeMap(_.copyToHost()).toArray, batch.numRows())
- }
- } else {
- batch
- }
-
- val rowIterator = hostBatch.rowIterator().asScala
-
- override def next: InternalRow = rowIterator.next
-
- override def hasNext: Boolean = rowIterator.hasNext
-
- }
- }
- }
-
- }
-
- val mapping = new mutable.HashMap[DataType, DataType]()
-
- def getSupportedDataType(curId: AtomicLong, dataType: DataType): DataType = {
- dataType match {
- case CalendarIntervalType =>
- intervalStructType
- case NullType =>
- ByteType
- case s: StructType =>
- val newStructType = StructType(
- s.indices.map { index =>
- StructField(curId.getAndIncrement().toString,
- getSupportedDataType(curId, s.fields(index).dataType), s.fields(index).nullable,
- s.fields(index).metadata)
- })
- mapping.put(s, newStructType)
- newStructType
- case a@ArrayType(elementType, nullable) =>
- val newArrayType =
- ArrayType(getSupportedDataType(curId, elementType), nullable)
- mapping.put(a, newArrayType)
- newArrayType
- case m@MapType(keyType, valueType, nullable) =>
- val newKeyType = getSupportedDataType(curId, keyType)
- val newValueType = getSupportedDataType(curId, valueType)
- val mapType = MapType(newKeyType, newValueType, nullable)
- mapping.put(m, mapType)
- mapType
- case d: DecimalType if d.scale < 0 =>
- val newType = if (d.precision <= Decimal.MAX_INT_DIGITS) {
- IntegerType
- } else {
- LongType
- }
- newType
- case _ =>
- dataType
- }
- }
-
- private def getSupportedSchemaFromUnsupported(
- cachedAttributes: Seq[Attribute],
- requestedAttributes: Seq[Attribute] = Seq.empty): (Seq[Attribute], Seq[Attribute]) = {
-
- // We only handle CalendarIntervalType, Decimals and NullType ATM convert it to a supported type
- val curId = new AtomicLong()
- val newCachedAttributes = cachedAttributes.map {
- attribute => val name = s"_col${curId.getAndIncrement()}"
- attribute.dataType match {
- case CalendarIntervalType =>
- AttributeReference(name, intervalStructType,
- attribute.nullable, metadata = attribute.metadata)(attribute.exprId)
- .asInstanceOf[Attribute]
- case NullType =>
- AttributeReference(name, DataTypes.ByteType,
- nullable = true, metadata =
- attribute.metadata)(attribute.exprId).asInstanceOf[Attribute]
- case StructType(_) | ArrayType(_, _) | MapType(_, _, _) | DecimalType() =>
- AttributeReference(name,
- getSupportedDataType(curId, attribute.dataType),
- attribute.nullable, attribute.metadata)(attribute.exprId)
- case _ =>
- attribute.withName(name)
- }
- }
-
- val newRequestedAttributes =
- getSelectedSchemaFromCachedSchema(requestedAttributes, newCachedAttributes)
-
- (newCachedAttributes, newRequestedAttributes)
- }
-
- private def getHadoopConf(requestedSchema: StructType,
- sqlConf: SQLConf): Configuration = {
-
- val hadoopConf = new Configuration(false)
- hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
- hadoopConf.set(
- ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- ParquetWriteSupport.SPARK_ROW_SCHEMA,
- requestedSchema.json)
- hadoopConf.set(
- SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sqlConf.sessionLocalTimeZone)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, false)
- hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, false)
-
- hadoopConf.set(ShimLoader.getSparkShims.parquetRebaseWriteKey,
- LegacyBehaviorPolicy.CORRECTED.toString)
-
- hadoopConf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key,
- SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString)
-
- hadoopConf.setBoolean(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, false)
-
- hadoopConf.set(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString)
-
- ParquetWriteSupport.setSchema(requestedSchema, hadoopConf)
-
- hadoopConf
- }
-
- /**
- * 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 rapidsConf = new RapidsConf(conf)
- val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
- val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
- if (rapidsConf.isSqlEnabled && isSchemaSupportedByCudf(schema)) {
- val structSchema = schemaWithUnambiguousNames.toStructType
- val converters = new GpuRowToColumnConverter(structSchema)
- val columnarBatchRdd = input.mapPartitions(iter => {
- new RowToColumnarIterator(iter, structSchema, RequireSingleBatch, converters)
- })
- columnarBatchRdd.flatMap(cb => {
- withResource(cb)(cb => compressColumnarBatchWithParquet(cb, structSchema,
- schema.toStructType, bytesAllowedPerBatch))
- })
- } else {
- val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
- input.mapPartitions {
- cbIter =>
- new CachedBatchIteratorProducer[InternalRow](cbIter, schemaWithUnambiguousNames, schema,
- broadcastedConf).getInternalRowToCachedBatchIterator
- }
- }
- }
-
- override def buildFilter(
- predicates: Seq[Expression],
- cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
- //essentially a noop
- (_: Int, b: Iterator[CachedBatch]) => b
- }
-}
-
-/**
- * Similar to ParquetFileFormat
- */
-private[rapids] class ParquetOutputFileFormat {
-
- def getRecordWriter(output: OutputFile, conf: Configuration): RecordWriter[Void, InternalRow] = {
- import ParquetOutputFormat._
-
- val blockSize = getLongBlockSize(conf)
- val maxPaddingSize =
- conf.getInt(MAX_PADDING_BYTES, ParquetWriter.MAX_PADDING_SIZE_DEFAULT)
- val validating = getValidation(conf)
-
- val writeSupport = new ParquetWriteSupport().asInstanceOf[WriteSupport[InternalRow]]
- val init = writeSupport.init(conf)
- val writer = new ParquetFileWriter(output, init.getSchema,
- Mode.CREATE, blockSize, maxPaddingSize)
- writer.start()
-
- val writerVersion =
- ParquetProperties.WriterVersion.fromString(conf.get(ParquetOutputFormat.WRITER_VERSION,
- ParquetProperties.WriterVersion.PARQUET_1_0.toString))
-
- val codecFactory = new CodecFactory(conf, getPageSize(conf))
-
- new ParquetRecordWriter[InternalRow](writer, writeSupport, init.getSchema,
- init.getExtraMetaData, blockSize, getPageSize(conf),
- codecFactory.getCompressor(CompressionCodecName.UNCOMPRESSED), getDictionaryPageSize(conf),
- getEnableDictionary(conf), validating, writerVersion,
- ParquetOutputFileFormat.getMemoryManager(conf))
-
- }
-}
-
-private object ParquetOutputFileFormat {
- var memoryManager: MemoryManager = _
- val DEFAULT_MEMORY_POOL_RATIO: Float = 0.95f
- val DEFAULT_MIN_MEMORY_ALLOCATION: Long = 1 * 1024 * 1024 // 1MB
-
- def getMemoryManager(conf: Configuration): MemoryManager = {
- synchronized {
- if (memoryManager == null) {
- import ParquetOutputFormat._
- val maxLoad = conf.getFloat(MEMORY_POOL_RATIO, DEFAULT_MEMORY_POOL_RATIO)
- val minAllocation = conf.getLong(MIN_MEMORY_ALLOCATION, DEFAULT_MIN_MEMORY_ALLOCATION)
- memoryManager = new MemoryManager(maxLoad, minAllocation)
- }
- }
- memoryManager
- }
-}
diff --git a/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/SparkBaseShims.scala b/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/SparkBaseShims.scala
index de63c4697d3..ab5fc1b7493 100644
--- a/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/SparkBaseShims.scala
+++ b/shims/spark313/src/main/scala/com/nvidia/spark/rapids/shims/spark313/SparkBaseShims.scala
@@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.shims.spark313
import java.net.URI
import java.nio.ByteBuffer
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.v2.Spark30XShims
import org.apache.arrow.memory.ReferenceManager
@@ -57,6 +58,7 @@ import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuB
import org.apache.spark.sql.rapids.execution.python.GpuPythonUDF
import org.apache.spark.sql.rapids.execution.python.shims.spark313._
import org.apache.spark.sql.rapids.shims.spark313._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
diff --git a/shims/spark313/src/main/scala/org/apache/spark/sql/rapids/shims/spark313/GpuInMemoryTableScanExec.scala b/shims/spark313/src/main/scala/org/apache/spark/sql/rapids/shims/spark313/GpuInMemoryTableScanExec.scala
deleted file mode 100644
index 25a116f3945..00000000000
--- a/shims/spark313/src/main/scala/org/apache/spark/sql/rapids/shims/spark313/GpuInMemoryTableScanExec.scala
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright (c) 2021, NVIDIA CORPORATION.
- *
- * 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 org.apache.spark.sql.rapids.shims.spark313
-
-import com.nvidia.spark.rapids.{GpuExec, GpuMetric}
-import com.nvidia.spark.rapids.shims.spark313.ParquetCachedBatchSerializer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder}
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.physical.Partitioning
-import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
-import org.apache.spark.sql.vectorized.ColumnarBatch
-
-case class GpuInMemoryTableScanExec(
- attributes: Seq[Attribute],
- predicates: Seq[Expression],
- @transient relation: InMemoryRelation) extends LeafExecNode with GpuExec {
-
- override val nodeName: String = {
- relation.cacheBuilder.tableName match {
- case Some(_) =>
- "Scan " + relation.cacheBuilder.cachedName
- case _ =>
- super.nodeName
- }
- }
-
- override def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren
-
- override def doCanonicalize(): SparkPlan =
- copy(attributes = attributes.map(QueryPlan.normalizeExpressions(_, relation.output)),
- predicates = predicates.map(QueryPlan.normalizeExpressions(_, relation.output)),
- relation = relation.canonicalized.asInstanceOf[InMemoryRelation])
-
- override def vectorTypes: Option[Seq[String]] =
- relation.cacheBuilder.serializer.vectorTypes(attributes, conf)
-
- private lazy val columnarInputRDD: RDD[ColumnarBatch] = {
- val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS)
- val buffers = filteredCachedBatches()
- relation.cacheBuilder.serializer.asInstanceOf[ParquetCachedBatchSerializer]
- .gpuConvertCachedBatchToColumnarBatch(
- buffers,
- relation.output,
- attributes,
- conf).map { cb =>
- numOutputRows += cb.numRows()
- cb
- }
- }
-
- override def output: Seq[Attribute] = attributes
-
- private def updateAttribute(expr: Expression): Expression = {
- // attributes can be pruned so using relation's output.
- // E.g., relation.output is [id, item] but this scan's output can be [item] only.
- val attrMap = AttributeMap(relation.cachedPlan.output.zip(relation.output))
- expr.transform {
- case attr: Attribute => attrMap.getOrElse(attr, attr)
- }
- }
-
- // The cached version does not change the outputPartitioning of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputPartitioning: Partitioning = {
- relation.cachedPlan.outputPartitioning match {
- case e: Expression => updateAttribute(e).asInstanceOf[Partitioning]
- case other => other
- }
- }
-
- // The cached version does not change the outputOrdering of the original SparkPlan.
- // But the cached version could alias output, so we need to replace output.
- override def outputOrdering: Seq[SortOrder] =
- relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder])
-
- lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled
-
- // Accumulators used for testing purposes
- lazy val readPartitions = sparkContext.longAccumulator
- lazy val readBatches = sparkContext.longAccumulator
-
- private def filteredCachedBatches() = {
- // Right now just return the batch without filtering
- relation.cacheBuilder.cachedColumnBuffers
- }
-
- protected override def doExecute(): RDD[InternalRow] = {
- throw new UnsupportedOperationException("This Exec only deals with Columnar Data")
- }
-
- protected override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- columnarInputRDD
- }
-}
diff --git a/shims/spark320/src/main/scala/com/nvidia/spark/rapids/shims/spark320/Spark320Shims.scala b/shims/spark320/src/main/scala/com/nvidia/spark/rapids/shims/spark320/Spark320Shims.scala
index b6d39037a08..4d84e9fd7bd 100644
--- a/shims/spark320/src/main/scala/com/nvidia/spark/rapids/shims/spark320/Spark320Shims.scala
+++ b/shims/spark320/src/main/scala/com/nvidia/spark/rapids/shims/spark320/Spark320Shims.scala
@@ -21,8 +21,9 @@ import java.nio.ByteBuffer
import scala.collection.mutable.ListBuffer
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids._
-import com.nvidia.spark.rapids.shims.v2._
+import com.nvidia.spark.rapids.shims.v2.Spark32XShims
import com.nvidia.spark.rapids.spark320.RapidsShuffleManager
import org.apache.arrow.memory.ReferenceManager
import org.apache.arrow.vector.ValueVector
@@ -58,6 +59,7 @@ import org.apache.spark.sql.rapids._
import org.apache.spark.sql.rapids.execution._
import org.apache.spark.sql.rapids.execution.python._
import org.apache.spark.sql.rapids.shims.v2._
+import org.apache.spark.sql.rapids.shims.v2.GpuInMemoryTableScanExec
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.storage.{BlockId, BlockManagerId}
diff --git a/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/ParquetCachedBatchSerializer.scala b/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/ParquetCachedBatchSerializer.scala
new file mode 100644
index 00000000000..0ba35718f87
--- /dev/null
+++ b/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/ParquetCachedBatchSerializer.scala
@@ -0,0 +1,197 @@
+/*
+ * Copyright (c) 2021, NVIDIA CORPORATION.
+ *
+ * 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
+
+import com.nvidia.spark.rapids.ShimLoader
+
+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.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.storage.StorageLevel
+
+
+trait GpuCachedBatchSerializer extends CachedBatchSerializer {
+ def gpuConvertCachedBatchToColumnarBatch(
+ input: RDD[CachedBatch],
+ cacheAttributes: Seq[Attribute],
+ selectedAttributes: Seq[Attribute],
+ conf: SQLConf): RDD[ColumnarBatch]
+}
+
+/**
+ * User facing wrapper class that calls into the proper shim version.
+ */
+class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer {
+
+ val minSupportedVer = "3.1.1"
+ val sparkVersion = ShimLoader.getSparkVersion
+ // Note that since the config to set the serializer wasn't added until
+ // Spark 3.1.0 (https://issues.apache.org/jira/browse/SPARK-32274) this shouldn't
+ // ever throw.
+ if (sparkVersion < minSupportedVer) {
+ throw new IllegalArgumentException("ParquetCachedBaatchSerializer only supported for Spark " +
+ s"versions > 3.1.1, version found was: $sparkVersion")
+ }
+
+ private lazy val realSerializer: GpuCachedBatchSerializer = {
+ ShimLoader.newInstanceOf("com.nvidia.spark.rapids.shims.v2.ParquetCachedBatchSerializer")
+ }
+
+ /**
+ * Can `convertColumnarBatchToCachedBatch()` be called instead of
+ * `convertInternalRowToCachedBatch()` for this given schema? True if it can and false if it
+ * cannot. Columnar input is only supported if the plan could produce columnar output. Currently
+ * this is mostly supported by input formats like parquet and orc, but more operations are likely
+ * to be supported soon.
+ * @param schema the schema of the data being stored.
+ * @return True if columnar input can be supported, else false.
+ */
+ override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = {
+ realSerializer.supportsColumnarInput(schema)
+ }
+
+ /**
+ * 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.
+ */
+ override def convertInternalRowToCachedBatch(
+ input: RDD[InternalRow],
+ schema: Seq[Attribute],
+ storageLevel: StorageLevel,
+ conf: SQLConf): RDD[CachedBatch] = {
+ realSerializer.convertInternalRowToCachedBatch(input, schema, storageLevel, conf)
+ }
+
+ /**
+ * 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.
+ */
+ override def convertColumnarBatchToCachedBatch(
+ input: RDD[ColumnarBatch],
+ schema: Seq[Attribute],
+ storageLevel: StorageLevel,
+ conf: SQLConf): RDD[CachedBatch] = {
+ realSerializer.convertColumnarBatchToCachedBatch(input, schema, storageLevel, conf)
+ }
+
+ /**
+ * 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.
+ */
+ override def buildFilter(
+ predicates: Seq[Expression],
+ cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
+ realSerializer.buildFilter(predicates, cachedAttributes)
+ }
+
+ /**
+ * Can `convertCachedBatchToColumnarBatch()` be called instead of
+ * `convertCachedBatchToInternalRow()` for this given schema? True if it can and false if it
+ * cannot. Columnar output is typically preferred because it is more efficient. Note that
+ * `convertCachedBatchToInternalRow()` must always be supported as there are other checks that
+ * can force row based output.
+ * @param schema the schema of the data being checked.
+ * @return true if columnar output should be used for this schema, else false.
+ */
+ override def supportsColumnarOutput(schema: StructType): Boolean = {
+ realSerializer.supportsColumnarOutput(schema)
+ }
+
+ /**
+ * Convert the cached data into a ColumnarBatch. This currently is only used if
+ * `supportsColumnarOutput()` returns true for the associated schema, but there are other checks
+ * that can force row based output. One of the main advantages of doing columnar output over row
+ * based output is that the code generation is more standard and can be combined with code
+ * generation for downstream operations.
+ * @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] = {
+ realSerializer.convertCachedBatchToColumnarBatch(input, cacheAttributes, selectedAttributes,
+ conf)
+ }
+
+ /**
+ * Convert the cached batch into `InternalRow`s. If you want this to be performant, code
+ * generation is advised.
+ * @param input the cached batches that should be 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] = {
+ realSerializer.convertCachedBatchToInternalRow(input, cacheAttributes, selectedAttributes, conf)
+ }
+
+ /**
+ * This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying 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 gpuConvertCachedBatchToColumnarBatch(
+ input: RDD[CachedBatch],
+ cacheAttributes: Seq[Attribute],
+ selectedAttributes: Seq[Attribute],
+ conf: SQLConf): RDD[ColumnarBatch] = {
+ realSerializer.gpuConvertCachedBatchToColumnarBatch(input, cacheAttributes,
+ selectedAttributes, conf)
+ }
+
+}
diff --git a/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala b/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala
index 44f880047e5..cea1864b121 100644
--- a/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala
+++ b/sql-plugin/src/main/311+-all/scala/com/nvidia/spark/rapids/shims/v2/ParquetCachedBatchSerializer.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import ai.rapids.cudf._
import ai.rapids.cudf.ParquetWriterOptions.StatisticsFrequency
+import com.nvidia.spark.GpuCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
import com.nvidia.spark.rapids.RapidsPluginImplicits._
@@ -258,7 +259,7 @@ private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) e
/**
* This class assumes, the data is Columnar and the plugin is on
*/
-class ParquetCachedBatchSerializer extends CachedBatchSerializer with Arm {
+class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer with Arm {
override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true
diff --git a/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/shims/v2/GpuInMemoryTableScanExec.scala b/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/shims/v2/GpuInMemoryTableScanExec.scala
index 7da80e47c79..a4261581983 100644
--- a/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/shims/v2/GpuInMemoryTableScanExec.scala
+++ b/sql-plugin/src/main/311+-all/scala/org/apache/spark/sql/rapids/shims/v2/GpuInMemoryTableScanExec.scala
@@ -16,8 +16,8 @@
package org.apache.spark.sql.rapids.shims.v2
+import com.nvidia.spark.ParquetCachedBatchSerializer
import com.nvidia.spark.rapids.{GpuExec, GpuMetric}
-import com.nvidia.spark.rapids.shims.v2.ParquetCachedBatchSerializer
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala
index 0ba51e05533..3b8d4d94c9a 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala
@@ -253,7 +253,7 @@ object ShimLoader extends Logging {
shimProviderClass = classname
}
- private def newInstanceOf[T](className: String): T = {
+ def newInstanceOf[T](className: String): T = {
val loader = getShimClassLoader()
logDebug(s"Loading $className using $loader with the parent loader ${loader.getParent}")
instantiateClass(loader.loadClass(className)).asInstanceOf[T]
@@ -303,4 +303,5 @@ object ShimLoader extends Logging {
def newUdfLogicalPlanRules(): Rule[LogicalPlan] = {
newInstanceOf("com.nvidia.spark.udf.LogicalPlanRules")
}
+
}