in shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala [192:298]
override def buildReaderWithPartitionValues(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
val sqlConf = sparkSession.sessionState.conf
val enableVectorizedReader = supportBatch(sparkSession, resultSchema)
val capacity = sqlConf.orcVectorizedReaderBatchSize
OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(hadoopConf, sqlConf.caseSensitiveAnalysis)
val broadcastedConf =
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
val orcFilterPushDown = sparkSession.sessionState.conf.orcFilterPushDown
val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles
(file: PartitionedFile) => {
val conf = broadcastedConf.value.value
val filePath = new Path(new URI(file.filePath))
val fs = filePath.getFileSystem(conf)
val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
val resultedColPruneInfo =
Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) {
reader =>
OrcUtils.requestedColumnIds(isCaseSensitive, dataSchema, requiredSchema, reader, conf)
}
if (resultedColPruneInfo.isEmpty) {
Iterator.empty
} else {
// ORC predicate pushdown
if (orcFilterPushDown && filters.nonEmpty) {
OrcUtils.readCatalystSchema(filePath, conf, ignoreCorruptFiles).foreach {
fileSchema =>
OrcFilters.createFilter(fileSchema, filters).foreach {
f => OrcInputFormat.setSearchArgument(conf, f, fileSchema.fieldNames)
}
}
}
val (requestedColIds, canPruneCols) = resultedColPruneInfo.get
val resultSchemaString = OrcUtils.orcResultSchemaString(
canPruneCols,
dataSchema,
resultSchema,
partitionSchema,
conf)
assert(
requestedColIds.length == requiredSchema.length,
"[BUG] requested column IDs do not match required schema")
val taskConf = new Configuration(conf)
val includeColumns = requestedColIds.filter(_ != -1).sorted.mkString(",")
taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, includeColumns)
val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty)
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId)
if (enableVectorizedReader) {
val batchReader = new OrcColumnarBatchReader(capacity)
// SPARK-23399 Register a task completion listener first to call `close()` in all cases.
// There is a possibility that `initialize` and `initBatch` hit some errors (like OOM)
// after opening a file.
val iter = new RecordReaderIterator(batchReader)
Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
val requestedDataColIds = requestedColIds ++ Array.fill(partitionSchema.length)(-1)
val requestedPartitionColIds =
Array.fill(requiredSchema.length)(-1) ++ Range(0, partitionSchema.length)
batchReader.initialize(fileSplit, taskAttemptContext)
batchReader.initBatch(
TypeDescription.fromString(resultSchemaString),
resultSchema.fields,
requestedDataColIds,
requestedPartitionColIds,
file.partitionValues)
iter.asInstanceOf[Iterator[InternalRow]]
} else {
val orcRecordReader = new OrcInputFormat[OrcStruct]
.createRecordReader(fileSplit, taskAttemptContext)
val iter = new RecordReaderIterator[OrcStruct](orcRecordReader)
Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
val deserializer = new OrcDeserializer(requiredSchema, requestedColIds)
if (partitionSchema.length == 0) {
iter.map(value => unsafeProjection(deserializer.deserialize(value)))
} else {
val joinedRow = new JoinedRow()
iter.map(
value =>
unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues)))
}
}
}
}
}