[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:
Reynold Xin 2016-04-26 10:58:56 -07:00 committed by Yin Huai
parent 92f66331b4
commit 5cb03220a0
12 changed files with 99 additions and 57 deletions

View file

@ -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 =>

View file

@ -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 =>

View file

@ -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)

View file

@ -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)
}

View file

@ -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")

View file

@ -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,

View file

@ -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()

View file

@ -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)

View file

@ -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

View file

@ -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

View file

@ -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")
}
}
}

View file

@ -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
}