[SPARK-14912][SQL] Propagate data source options to Hadoop configuration
## What changes were proposed in this pull request? We currently have no way for users to propagate options to the underlying library that rely in Hadoop configurations to work. For example, there are various options in parquet-mr that users might want to set, but the data source API does not expose a per-job way to set it. This patch propagates the user-specified options also into Hadoop Configuration. ## How was this patch tested? Used a mock data source implementation to test both the read path and the write path. Author: Reynold Xin <rxin@databricks.com> Closes #12688 from rxin/SPARK-14912.
This commit is contained in:
parent
92f66331b4
commit
5cb03220a0
|
@ -181,20 +181,20 @@ class DefaultSource extends FileFormat with DataSourceRegister {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
verifySchema(dataSchema)
|
||||
val numFeatures = options("numFeatures").toInt
|
||||
assert(numFeatures > 0)
|
||||
|
||||
val sparse = options.getOrElse("vectorType", "sparse") == "sparse"
|
||||
|
||||
val broadcastedConf = sparkSession.sparkContext.broadcast(
|
||||
new SerializableConfiguration(
|
||||
new Configuration(sparkSession.sparkContext.hadoopConfiguration)))
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
val points =
|
||||
new HadoopFileLinesReader(file, broadcastedConf.value.value)
|
||||
new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value)
|
||||
.map(_.toString.trim)
|
||||
.filterNot(line => line.isEmpty || line.startsWith("#"))
|
||||
.map { line =>
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources
|
|||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path}
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
|
@ -106,13 +107,17 @@ private[sql] object FileSourceStrategy extends Strategy with Logging {
|
|||
val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter)
|
||||
logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}")
|
||||
|
||||
val hadoopConf = new Configuration(files.sparkSession.sessionState.hadoopConf)
|
||||
files.options.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) }
|
||||
|
||||
val readFile = files.fileFormat.buildReader(
|
||||
sparkSession = files.sparkSession,
|
||||
dataSchema = files.dataSchema,
|
||||
partitionSchema = files.partitionSchema,
|
||||
requiredSchema = prunedDataSchema,
|
||||
filters = pushedDownFilters,
|
||||
options = files.options)
|
||||
options = files.options,
|
||||
hadoopConf = hadoopConf)
|
||||
|
||||
val plannedPartitions = files.bucketSpec match {
|
||||
case Some(bucketing) if files.sparkSession.sessionState.conf.bucketingEnabled =>
|
||||
|
|
|
@ -106,6 +106,10 @@ private[sql] case class InsertIntoHadoopFsRelation(
|
|||
val job = Job.getInstance(hadoopConf)
|
||||
job.setOutputKeyClass(classOf[Void])
|
||||
job.setOutputValueClass(classOf[InternalRow])
|
||||
|
||||
// Also set the options in Hadoop Configuration
|
||||
options.foreach { case (k, v) => if (v ne null) job.getConfiguration.set(k, v) }
|
||||
|
||||
FileOutputFormat.setOutputPath(job, qualifiedOutputPath)
|
||||
|
||||
val partitionSet = AttributeSet(partitionColumns)
|
||||
|
|
|
@ -99,16 +99,17 @@ class DefaultSource extends FileFormat with DataSourceRegister {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
val csvOptions = new CSVOptions(options)
|
||||
val headers = requiredSchema.fields.map(_.name)
|
||||
|
||||
val conf = new Configuration(sparkSession.sessionState.hadoopConf)
|
||||
val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(conf))
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
val lineIterator = {
|
||||
val conf = broadcastedConf.value.value
|
||||
val conf = broadcastedHadoopConf.value.value
|
||||
new HadoopFileLinesReader(file, conf).map { line =>
|
||||
new String(line.getBytes, 0, line.getLength, csvOptions.charset)
|
||||
}
|
||||
|
|
|
@ -217,7 +217,8 @@ trait FileFormat {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = {
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
// TODO: Remove this default implementation when the other formats have been ported
|
||||
// Until then we guard in [[FileSourceStrategy]] to only call this method on supported formats.
|
||||
throw new UnsupportedOperationException(s"buildReader is not supported for $this")
|
||||
|
|
|
@ -97,10 +97,10 @@ class DefaultSource extends FileFormat with DataSourceRegister {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = {
|
||||
val conf = new Configuration(sparkSession.sessionState.hadoopConf)
|
||||
val broadcastedConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(conf))
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
val parsedOptions: JSONOptions = new JSONOptions(options)
|
||||
val columnNameOfCorruptRecord = parsedOptions.columnNameOfCorruptRecord
|
||||
|
@ -109,8 +109,8 @@ class DefaultSource extends FileFormat with DataSourceRegister {
|
|||
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
|
||||
val joinedRow = new JoinedRow()
|
||||
|
||||
file => {
|
||||
val lines = new HadoopFileLinesReader(file, broadcastedConf.value.value).map(_.toString)
|
||||
(file: PartitionedFile) => {
|
||||
val lines = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map(_.toString)
|
||||
|
||||
val rows = JacksonParser.parseJson(
|
||||
lines,
|
||||
|
|
|
@ -262,13 +262,13 @@ private[sql] class DefaultSource
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = {
|
||||
val parquetConf = new Configuration(sparkSession.sessionState.hadoopConf)
|
||||
parquetConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[CatalystReadSupport].getName)
|
||||
parquetConf.set(
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[CatalystReadSupport].getName)
|
||||
hadoopConf.set(
|
||||
CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
|
||||
CatalystSchemaConverter.checkFieldNames(requiredSchema).json)
|
||||
parquetConf.set(
|
||||
hadoopConf.set(
|
||||
CatalystWriteSupport.SPARK_ROW_SCHEMA,
|
||||
CatalystSchemaConverter.checkFieldNames(requiredSchema).json)
|
||||
|
||||
|
@ -276,13 +276,13 @@ private[sql] class DefaultSource
|
|||
// This metadata is only useful for detecting optional columns when pushdowning filters.
|
||||
val dataSchemaToWrite = StructType.removeMetadata(StructType.metadataKeyForOptionalField,
|
||||
requiredSchema).asInstanceOf[StructType]
|
||||
CatalystWriteSupport.setSchema(dataSchemaToWrite, parquetConf)
|
||||
CatalystWriteSupport.setSchema(dataSchemaToWrite, hadoopConf)
|
||||
|
||||
// Sets flags for `CatalystSchemaConverter`
|
||||
parquetConf.setBoolean(
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_BINARY_AS_STRING.key,
|
||||
sparkSession.getConf(SQLConf.PARQUET_BINARY_AS_STRING))
|
||||
parquetConf.setBoolean(
|
||||
hadoopConf.setBoolean(
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
|
||||
sparkSession.getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP))
|
||||
|
||||
|
@ -298,8 +298,8 @@ private[sql] class DefaultSource
|
|||
None
|
||||
}
|
||||
|
||||
val broadcastedConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(parquetConf))
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
// TODO: if you move this into the closure it reverts to the default values.
|
||||
// If true, enable using the custom RecordReader for parquet. This only works for
|
||||
|
@ -327,7 +327,8 @@ private[sql] class DefaultSource
|
|||
null)
|
||||
|
||||
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
|
||||
val hadoopAttemptContext = new TaskAttemptContextImpl(broadcastedConf.value.value, attemptId)
|
||||
val hadoopAttemptContext =
|
||||
new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId)
|
||||
|
||||
val parquetReader = if (enableVectorizedReader) {
|
||||
val vectorizedReader = new VectorizedParquetRecordReader()
|
||||
|
|
|
@ -89,17 +89,17 @@ class DefaultSource extends FileFormat with DataSourceRegister {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = {
|
||||
val conf = new Configuration(sparkSession.sessionState.hadoopConf)
|
||||
val broadcastedConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(conf))
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
file => {
|
||||
(file: PartitionedFile) => {
|
||||
val unsafeRow = new UnsafeRow(1)
|
||||
val bufferHolder = new BufferHolder(unsafeRow)
|
||||
val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1)
|
||||
|
||||
new HadoopFileLinesReader(file, broadcastedConf.value.value).map { line =>
|
||||
new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { line =>
|
||||
// Writes to an UnsafeRow directly
|
||||
bufferHolder.reset()
|
||||
unsafeRowWriter.write(0, line.getBytes, 0, line.getLength)
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources
|
|||
import java.io.File
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{BlockLocation, FileStatus, RawLocalFileSystem}
|
||||
import org.apache.hadoop.mapreduce.Job
|
||||
|
||||
|
@ -476,7 +477,8 @@ class TestFileFormat extends FileFormat {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = {
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
|
||||
|
||||
// Record the arguments so they can be checked in the test case.
|
||||
LastArguments.partitionSchema = partitionSchema
|
||||
|
|
|
@ -114,22 +114,21 @@ private[sql] class DefaultSource
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
val orcConf = new Configuration(sparkSession.sessionState.hadoopConf)
|
||||
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
if (sparkSession.sessionState.conf.orcFilterPushDown) {
|
||||
// Sets pushed predicates
|
||||
OrcFilters.createFilter(filters.toArray).foreach { f =>
|
||||
orcConf.set(OrcTableScan.SARG_PUSHDOWN, f.toKryo)
|
||||
orcConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true)
|
||||
hadoopConf.set(OrcTableScan.SARG_PUSHDOWN, f.toKryo)
|
||||
hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true)
|
||||
}
|
||||
}
|
||||
|
||||
val broadcastedConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(orcConf))
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
val conf = broadcastedConf.value.value
|
||||
val conf = broadcastedHadoopConf.value.value
|
||||
|
||||
// SPARK-8501: Empty ORC files always have an empty schema stored in their footer. In this
|
||||
// case, `OrcFileOperator.readSchema` returns `None`, and we can't read the underlying file
|
||||
|
@ -154,7 +153,7 @@ private[sql] class DefaultSource
|
|||
// Specifically would be helpful for partitioned datasets.
|
||||
val orcReader = OrcFile.createReader(
|
||||
new Path(new URI(file.filePath)), OrcFile.readerOptions(conf))
|
||||
new SparkOrcNewRecordReader(orcReader, conf, fileSplit.getStart(), fileSplit.getLength())
|
||||
new SparkOrcNewRecordReader(orcReader, conf, fileSplit.getStart, fileSplit.getLength)
|
||||
}
|
||||
|
||||
// Unwraps `OrcStruct`s to `UnsafeRow`s
|
||||
|
|
|
@ -65,4 +65,27 @@ class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest with Predicat
|
|||
.load(file.getCanonicalPath))
|
||||
}
|
||||
}
|
||||
|
||||
test("test hadoop conf option propagation") {
|
||||
withTempPath { file =>
|
||||
// Test write side
|
||||
val df = sqlContext.range(10).selectExpr("cast(id as string)")
|
||||
df.write
|
||||
.option("some-random-write-option", "hahah-WRITE")
|
||||
.option("some-null-value-option", null) // test null robustness
|
||||
.option("dataSchema", df.schema.json)
|
||||
.format(dataSourceName).save(file.getAbsolutePath)
|
||||
assert(SimpleTextRelation.lastHadoopConf.get.get("some-random-write-option") == "hahah-WRITE")
|
||||
|
||||
// Test read side
|
||||
val df1 = sqlContext.read
|
||||
.option("some-random-read-option", "hahah-READ")
|
||||
.option("some-null-value-option", null) // test null robustness
|
||||
.option("dataSchema", df.schema.json)
|
||||
.format(dataSourceName)
|
||||
.load(file.getAbsolutePath)
|
||||
df1.count()
|
||||
assert(SimpleTextRelation.lastHadoopConf.get.get("some-random-read-option") == "hahah-READ")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,13 +47,16 @@ class SimpleTextSource extends FileFormat with DataSourceRegister {
|
|||
sparkSession: SparkSession,
|
||||
job: Job,
|
||||
options: Map[String, String],
|
||||
dataSchema: StructType): OutputWriterFactory = new OutputWriterFactory {
|
||||
override def newInstance(
|
||||
path: String,
|
||||
bucketId: Option[Int],
|
||||
dataSchema: StructType,
|
||||
context: TaskAttemptContext): OutputWriter = {
|
||||
new SimpleTextOutputWriter(path, context)
|
||||
dataSchema: StructType): OutputWriterFactory = {
|
||||
SimpleTextRelation.lastHadoopConf = Option(job.getConfiguration)
|
||||
new OutputWriterFactory {
|
||||
override def newInstance(
|
||||
path: String,
|
||||
bucketId: Option[Int],
|
||||
dataSchema: StructType,
|
||||
context: TaskAttemptContext): OutputWriter = {
|
||||
new SimpleTextOutputWriter(path, context)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -63,8 +66,9 @@ class SimpleTextSource extends FileFormat with DataSourceRegister {
|
|||
partitionSchema: StructType,
|
||||
requiredSchema: StructType,
|
||||
filters: Seq[Filter],
|
||||
options: Map[String, String]): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
|
||||
options: Map[String, String],
|
||||
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
|
||||
SimpleTextRelation.lastHadoopConf = Option(hadoopConf)
|
||||
SimpleTextRelation.requiredColumns = requiredSchema.fieldNames
|
||||
SimpleTextRelation.pushedFilters = filters.toSet
|
||||
|
||||
|
@ -74,9 +78,8 @@ class SimpleTextSource extends FileFormat with DataSourceRegister {
|
|||
inputAttributes.find(_.name == field.name)
|
||||
}
|
||||
|
||||
val conf = new Configuration(sparkSession.sessionState.hadoopConf)
|
||||
val broadcastedConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(conf))
|
||||
val broadcastedHadoopConf =
|
||||
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
|
||||
|
||||
(file: PartitionedFile) => {
|
||||
val predicate = {
|
||||
|
@ -95,7 +98,7 @@ class SimpleTextSource extends FileFormat with DataSourceRegister {
|
|||
val projection = new InterpretedProjection(outputAttributes, inputAttributes)
|
||||
|
||||
val unsafeRowIterator =
|
||||
new HadoopFileLinesReader(file, broadcastedConf.value.value).map { line =>
|
||||
new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { line =>
|
||||
val record = line.toString
|
||||
new GenericInternalRow(record.split(",", -1).zip(fieldTypes).map {
|
||||
case (v, dataType) =>
|
||||
|
@ -164,4 +167,7 @@ object SimpleTextRelation {
|
|||
|
||||
// Used to test failure callback
|
||||
var callbackCalled = false
|
||||
|
||||
// Used by the test case to check the value propagated in the hadoop confs.
|
||||
var lastHadoopConf: Option[Configuration] = None
|
||||
}
|
||||
|
|
Loading…
Reference in a new issue