Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Fix reading ORC/PARQUET over empty clipped schema #5674

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 11 additions & 0 deletions integration_tests/src/main/python/orc_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,17 @@ def test_merge_schema_read(spark_tmp_path, v1_enabled_list, reader_confs):
lambda spark : spark.read.option('mergeSchema', 'true').orc(data_path),
conf=all_confs)

@pytest.mark.parametrize('v1_enabled_list', ["", "orc"])
@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn)
def test_read_orc_with_empty_clipped_schema(spark_tmp_path, v1_enabled_list, reader_confs):
data_path = spark_tmp_path + '/ORC_DATA'
with_cpu_session(
lambda spark: gen_df(spark, [('a', int_gen)], length=100).write.orc(data_path))
schema = StructType([StructField('b', IntegerType()), StructField('c', StringType())])
all_confs = copy_and_update(reader_confs, {'spark.sql.sources.useV1SourceList': v1_enabled_list})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: spark.read.schema(schema).orc(data_path), conf=all_confs)

@pytest.mark.parametrize('v1_enabled_list', ["", "orc"])
@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn)
def test_orc_read_multiple_schema(spark_tmp_path, v1_enabled_list, reader_confs):
Expand Down
11 changes: 11 additions & 0 deletions integration_tests/src/main/python/parquet_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -418,6 +418,17 @@ def test_parquet_read_merge_schema_native_fallback(spark_tmp_path, v1_enabled_li
cpu_fallback_class_name='FileSourceScanExec' if v1_enabled_list == 'parquet' else 'BatchScanExec',
conf=all_confs)

@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"])
@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn)
def test_read_parquet_with_empty_clipped_schema(spark_tmp_path, v1_enabled_list, reader_confs):
data_path = spark_tmp_path + '/PARQUET_DATA'
with_cpu_session(
lambda spark: gen_df(spark, [('a', int_gen)], length=100).write.parquet(data_path))
schema = StructType([StructField('b', IntegerType()), StructField('c', StringType())])
all_confs = copy_and_update(reader_confs, {'spark.sql.sources.useV1SourceList': v1_enabled_list})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: spark.read.schema(schema).parquet(data_path), conf=all_confs)

@pytest.mark.parametrize('reader_confs', reader_opt_confs)
@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"])
def test_parquet_input_meta(spark_tmp_path, v1_enabled_list, reader_confs):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import scala.math.max

import ai.rapids.cudf.{ColumnVector, HostMemoryBuffer, NvtxColor, NvtxRange, Table}
import com.nvidia.spark.rapids.GpuMetric.{NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, SEMAPHORE_WAIT_TIME}
import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq
import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
Expand All @@ -43,7 +44,7 @@ import org.apache.spark.sql.rapids.InputFileUtils
import org.apache.spark.sql.rapids.execution.TrampolineUtil
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector => SparkVector}
import org.apache.spark.util.SerializableConfiguration

/**
Expand Down Expand Up @@ -542,7 +543,9 @@ trait DataBlockBase {
*
* The sub-class should wrap the real schema for the specific file format
*/
trait SchemaBase
trait SchemaBase {
def fieldNames: Array[String]
}

/**
* A common trait for the extra information for different file format
Expand Down Expand Up @@ -793,14 +796,17 @@ abstract class MultiFileCoalescingPartitionReaderBase(
private def readBatch(): Option[ColumnarBatch] = {
withResource(new NvtxRange(s"$getFileFormatShortName readBatch", NvtxColor.GREEN)) { _ =>
val currentChunkMeta = populateCurrentBlockChunk()
if (readDataSchema.isEmpty) {
if (currentChunkMeta.clippedSchema.fieldNames.isEmpty) {
// not reading any data, so return a degenerate ColumnarBatch with the row count
if (currentChunkMeta.numTotalRows == 0) {
None
} else {
val rows = currentChunkMeta.numTotalRows.toInt
// Someone is going to process this data, even if it is just a row count
GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME))
Copy link
Member

Choose a reason for hiding this comment

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

This is inconsistent with the semantics of the ORC scan below. In this case, it will acquire the semaphore despite returning a row-count-only batch, while the ORC case below will avoid acquiring the semaphore unless it actually generates null columns.

Copy link
Collaborator Author

@sperlingxx sperlingxx May 30, 2022

Choose a reason for hiding this comment

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

Aligned all GPU semaphore acquisitions for empty batches.

val emptyBatch = new ColumnarBatch(Array.empty, currentChunkMeta.numTotalRows.toInt)
val nullColumns = readDataSchema.safeMap(f =>
GpuColumnVector.fromNull(rows, f.dataType).asInstanceOf[SparkVector])
val emptyBatch = new ColumnarBatch(nullColumns.toArray, rows)
addAllPartitionValues(Some(emptyBatch), currentChunkMeta.allPartValues,
currentChunkMeta.rowsPerPartition, partitionSchema)
}
Expand Down
76 changes: 52 additions & 24 deletions sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, MapType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector => SparkVector}
import org.apache.spark.util.SerializableConfiguration

case class GpuOrcScan(
Expand Down Expand Up @@ -664,10 +664,18 @@ class GpuOrcPartitionReader(
withResource(new NvtxRange("ORC readBatch", NvtxColor.GREEN)) { _ =>
val currentStripes = populateCurrentBlockChunk(ctx.blockIterator, maxReadBatchSizeRows,
maxReadBatchSizeBytes)
if (readDataSchema.isEmpty) {
if (ctx.updatedReadSchema.isEmpty) {
// not reading any data, so return a degenerate ColumnarBatch with the row count
val numRows = currentStripes.map(_.infoBuilder.getNumberOfRows).sum
Some(new ColumnarBatch(Array.empty, numRows.toInt))
val numRows = currentStripes.map(_.infoBuilder.getNumberOfRows).sum.toInt
if (numRows == 0) {
None
} else {
// Someone is going to process this data, even if it is just a row count
GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME))
val nullColumns = readDataSchema.safeMap(f =>
GpuColumnVector.fromNull(numRows, f.dataType).asInstanceOf[SparkVector])
Some(new ColumnarBatch(nullColumns.toArray, numRows))
}
} else {
val table = readToTable(currentStripes)
try {
Expand Down Expand Up @@ -1389,13 +1397,25 @@ class MultiFileCloudOrcPartitionReader(
extends MultiFileCloudPartitionReaderBase(conf, files, numThreads, maxNumFileProcessed, filters,
execMetrics, ignoreCorruptFiles) with MultiFileReaderFunctions with OrcPartitionReaderBase {

private case class HostMemoryEmptyMetaData(
override val partitionedFile: PartitionedFile,
bufferSize: Long,
override val bytesRead: Long,
updatedReadSchema: TypeDescription,
readSchema: StructType) extends HostMemoryBuffersWithMetaDataBase {

override def memBuffersAndSizes: Array[(HostMemoryBuffer, Long)] =
Array(null.asInstanceOf[HostMemoryBuffer] -> bufferSize)
}

private case class HostMemoryBuffersWithMetaData(
override val partitionedFile: PartitionedFile,
override val memBuffersAndSizes: Array[(HostMemoryBuffer, Long)],
override val bytesRead: Long,
updatedReadSchema: TypeDescription,
requestedMapping: Option[Array[Int]]) extends HostMemoryBuffersWithMetaDataBase


private class ReadBatchRunner(
taskContext: TaskContext,
partFile: PartitionedFile,
Expand All @@ -1411,13 +1431,13 @@ class MultiFileCloudOrcPartitionReader(
} catch {
case e: FileNotFoundException if ignoreMissingFiles =>
logWarning(s"Skipped missing file: ${partFile.filePath}", e)
HostMemoryBuffersWithMetaData(partFile, Array((null, 0)), 0, null, None)
HostMemoryEmptyMetaData(partFile, 0, 0, null, null)
// Throw FileNotFoundException even if `ignoreCorruptFiles` is true
case e: FileNotFoundException if !ignoreMissingFiles => throw e
case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
logWarning(
s"Skipped the rest of the content in the corrupted file: ${partFile.filePath}", e)
HostMemoryBuffersWithMetaData(partFile, Array((null, 0)), 0, null, None)
HostMemoryEmptyMetaData(partFile, 0, 0, null, null)
} finally {
TrampolineUtil.unsetTaskContext()
}
Expand All @@ -1433,22 +1453,21 @@ class MultiFileCloudOrcPartitionReader(
if (ctx == null || ctx.blockIterator.isEmpty) {
val bytesRead = fileSystemBytesRead() - startingBytesRead
// no blocks so return null buffer and size 0
return HostMemoryBuffersWithMetaData(partFile, Array((null, 0)), bytesRead,
ctx.updatedReadSchema, ctx.requestedMapping)
return HostMemoryEmptyMetaData(partFile, 0, bytesRead,
ctx.updatedReadSchema, readDataSchema)
}
blockChunkIter = ctx.blockIterator
if (isDone) {
val bytesRead = fileSystemBytesRead() - startingBytesRead
// got close before finishing
HostMemoryBuffersWithMetaData(partFile, Array((null, 0)), bytesRead,
ctx.updatedReadSchema, ctx.requestedMapping)
HostMemoryEmptyMetaData(partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema)
} else {
if (readDataSchema.isEmpty) {
if (ctx.updatedReadSchema.isEmpty) {
val bytesRead = fileSystemBytesRead() - startingBytesRead
val numRows = ctx.blockIterator.map(_.infoBuilder.getNumberOfRows).sum.toInt
// overload size to be number of rows with null buffer
HostMemoryBuffersWithMetaData(partFile, Array((null, numRows)), bytesRead,
ctx.updatedReadSchema, ctx.requestedMapping)
HostMemoryEmptyMetaData(partFile, numRows, bytesRead,
ctx.updatedReadSchema, readDataSchema)
} else {
while (blockChunkIter.hasNext) {
val blocksToRead = populateCurrentBlockChunk(blockChunkIter, maxReadBatchSizeRows,
Expand All @@ -1459,8 +1478,7 @@ class MultiFileCloudOrcPartitionReader(
if (isDone) {
// got close before finishing
hostBuffers.foreach(_._1.safeClose())
HostMemoryBuffersWithMetaData(partFile, Array((null, 0)), bytesRead,
ctx.updatedReadSchema, ctx.requestedMapping)
HostMemoryEmptyMetaData(partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema)
} else {
HostMemoryBuffersWithMetaData(partFile, hostBuffers.toArray, bytesRead,
ctx.updatedReadSchema, ctx.requestedMapping)
Expand Down Expand Up @@ -1524,6 +1542,20 @@ class MultiFileCloudOrcPartitionReader(
override def readBatch(fileBufsAndMeta: HostMemoryBuffersWithMetaDataBase):
Option[ColumnarBatch] = {
fileBufsAndMeta match {
case meta: HostMemoryEmptyMetaData =>
// Not reading any data, but add in partition data if needed
val rows = meta.bufferSize.toInt
val batch = if (rows == 0) {
new ColumnarBatch(Array.empty, 0)
} else {
// Someone is going to process this data, even if it is just a row count
GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME))
val nullColumns = meta.readSchema.fields.safeMap(f =>
GpuColumnVector.fromNull(rows, f.dataType).asInstanceOf[SparkVector])
new ColumnarBatch(nullColumns, rows)
}
addPartitionValues(Some(batch), meta.partitionedFile.partitionValues, partitionSchema)

case buffer: HostMemoryBuffersWithMetaData =>
val memBuffersAndSize = buffer.memBuffersAndSizes
val (hostBuffer, size) = memBuffersAndSize.head
Expand All @@ -1536,6 +1568,7 @@ class MultiFileCloudOrcPartitionReader(
currentFileHostBuffers = None
}
nextBatch

case _ => throw new RuntimeException("Wrong HostMemoryBuffersWithMetaData")
}
}
Expand All @@ -1548,14 +1581,6 @@ class MultiFileCloudOrcPartitionReader(
fileName: String,
updatedReadSchema: TypeDescription,
requestedMapping: Option[Array[Int]] = None): Option[ColumnarBatch] = {
// Not reading any data, but add in partition data if needed
if (hostBuffer == null) {
// Someone is going to process this data, even if it is just a row count
GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME))
val emptyBatch = new ColumnarBatch(Array.empty, dataSize.toInt)
return addPartitionValues(Some(emptyBatch), partValues, partitionSchema)
}

val table = withResource(hostBuffer) { _ =>
// Dump ORC data into a file
dumpDataToFile(hostBuffer, dataSize, files, Option(debugDumpPrefix), Some("orc"))
Expand Down Expand Up @@ -1637,7 +1662,10 @@ trait OrcCodecWritingHelper extends Arm {
}

// Orc schema wrapper
private case class OrcSchemaWrapper(schema: TypeDescription) extends SchemaBase
private case class OrcSchemaWrapper(schema: TypeDescription) extends SchemaBase {

override def fieldNames: Array[String] = schema.getFieldNames.asScala.toArray
}

case class OrcStripeWithMeta(stripe: OrcOutputStripe, ctx: OrcPartitionReaderContext)
// OrcOutputStripe wrapper
Expand Down
Loading